This is an automated email from the ASF dual-hosted git repository.
benjobs pushed a commit to branch dev
in repository https://gitbox.apache.org/repos/asf/incubator-streampark.git
The following commit(s) were added to refs/heads/dev by this push:
new 74745301a [Feature] Simplify ClusterClientTrait (#1741)
74745301a is described below
commit 74745301aa59a0c8b326aae97df1289e316f47e5
Author: 1996fanrui <[email protected]>
AuthorDate: Thu Oct 6 01:07:30 2022 +0800
[Feature] Simplify ClusterClientTrait (#1741)
---
.../streampark/flink/core/ClusterClientTrait.scala | 60 ----------------------
.../streampark/flink/core/FlinkClientTrait.scala} | 14 ++---
.../flink/core/FlinkClusterClient.scala} | 12 +----
.../flink/core/FlinkClusterClient.scala} | 12 +----
.../flink/core/FlinkClusterClient.scala} | 12 +----
.../streampark/flink/core/ClusterClient.scala | 51 ------------------
.../flink/core/FlinkClusterClient.scala} | 16 +++---
.../flink/submit/trait/FlinkSubmitTrait.scala | 8 +--
8 files changed, 28 insertions(+), 157 deletions(-)
diff --git
a/streampark-flink/streampark-flink-shims/streampark-flink-shims-base/src/main/scala/org/apache/streampark/flink/core/ClusterClientTrait.scala
b/streampark-flink/streampark-flink-shims/streampark-flink-shims-base/src/main/scala/org/apache/streampark/flink/core/ClusterClientTrait.scala
deleted file mode 100644
index 0a3729e68..000000000
---
a/streampark-flink/streampark-flink-shims/streampark-flink-shims-base/src/main/scala/org/apache/streampark/flink/core/ClusterClientTrait.scala
+++ /dev/null
@@ -1,60 +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.streampark.flink.core
-
-import org.apache.flink.api.common.{JobID, JobStatus}
-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, OperatorID}
-import org.apache.flink.runtime.jobmaster.JobResult
-import org.apache.flink.runtime.messages.Acknowledge
-import org.apache.flink.runtime.operators.coordination.{CoordinationRequest,
CoordinationResponse}
-
-import java.util
-import java.util.concurrent.CompletableFuture
-
-abstract class ClusterClientTrait[T](clusterClient: ClusterClient[T]) extends
ClusterClient[T] {
-
- override def close(): Unit = clusterClient.close()
-
- override def getClusterId = clusterClient.getClusterId
-
- override def getFlinkConfiguration: Configuration =
clusterClient.getFlinkConfiguration
-
- override def shutDownCluster(): Unit = clusterClient.shutDownCluster()
-
- override def getWebInterfaceURL: String = clusterClient.getWebInterfaceURL
-
- override def listJobs():
CompletableFuture[util.Collection[JobStatusMessage]] = clusterClient.listJobs()
-
- override def disposeSavepoint(s: String): CompletableFuture[Acknowledge] =
clusterClient.disposeSavepoint(s)
-
- override def submitJob(jobGraph: JobGraph): CompletableFuture[JobID] =
clusterClient.submitJob(jobGraph)
-
- override def getJobStatus(jobID: JobID): CompletableFuture[JobStatus] =
clusterClient.getJobStatus(jobID)
-
- override def requestJobResult(jobID: JobID): CompletableFuture[JobResult] =
clusterClient.requestJobResult(jobID)
-
- override def getAccumulators(jobID: JobID, classLoader: ClassLoader):
CompletableFuture[util.Map[String, AnyRef]] =
clusterClient.getAccumulators(jobID, classLoader)
-
- override def cancel(jobID: JobID): CompletableFuture[Acknowledge] =
clusterClient.cancel(jobID)
-
- override def sendCoordinationRequest(jobID: JobID, operatorID: OperatorID,
coordinationRequest: CoordinationRequest):
CompletableFuture[CoordinationResponse] =
clusterClient.sendCoordinationRequest(jobID, operatorID, coordinationRequest)
-}
-
diff --git
a/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.14/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
b/streampark-flink/streampark-flink-shims/streampark-flink-shims-base/src/main/scala/org/apache/streampark/flink/core/FlinkClientTrait.scala
similarity index 61%
rename from
streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.14/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
rename to
streampark-flink/streampark-flink-shims/streampark-flink-shims-base/src/main/scala/org/apache/streampark/flink/core/FlinkClientTrait.scala
index c4ff2d99e..4a8144b12 100644
---
a/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.14/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
+++
b/streampark-flink/streampark-flink-shims/streampark-flink-shims-base/src/main/scala/org/apache/streampark/flink/core/FlinkClientTrait.scala
@@ -14,19 +14,21 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.streampark.flink.core
import org.apache.flink.api.common.JobID
-import org.apache.flink.client.program.{ClusterClient => FlinkClusterClient}
+import org.apache.flink.client.program.ClusterClient
import java.util.concurrent.CompletableFuture
-class ClusterClient[T](clusterClient: FlinkClusterClient[T]) extends
ClusterClientTrait[T](clusterClient) {
-
- override def cancelWithSavepoint(jobID: JobID, s: String):
CompletableFuture[String] = clusterClient.cancelWithSavepoint(jobID, s)
+abstract class FlinkClientTrait[T](clusterClient: ClusterClient[T]) {
- override def stopWithSavepoint(jobID: JobID, b: Boolean, s: String):
CompletableFuture[String] = clusterClient.stopWithSavepoint(jobID, b, s)
+ def cancelWithSavepoint(jobID: JobID, s: String): CompletableFuture[String]
= {
+ clusterClient.cancelWithSavepoint(jobID, s)
+ }
- override def triggerSavepoint(jobID: JobID, s: String):
CompletableFuture[String] = clusterClient.triggerSavepoint(jobID, s)
+ def stopWithSavepoint(jobID: JobID, b: Boolean, s: String):
CompletableFuture[String] = clusterClient.stopWithSavepoint(jobID, b, s)
}
+
diff --git
a/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.13/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
b/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.12/src/main/scala/org/apache/streampark/flink/core/FlinkClusterClient.scala
similarity index 55%
rename from
streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.13/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
rename to
streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.12/src/main/scala/org/apache/streampark/flink/core/FlinkClusterClient.scala
index eed165104..6549e5e24 100644
---
a/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.13/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
+++
b/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.12/src/main/scala/org/apache/streampark/flink/core/FlinkClusterClient.scala
@@ -15,17 +15,9 @@
* limitations under the License.
*/
package org.apache.streampark.flink.core
-import org.apache.flink.api.common.JobID
-import org.apache.flink.client.program.{ClusterClient => FlinkClusterClient}
-import java.util.concurrent.CompletableFuture
+import org.apache.flink.client.program.ClusterClient
-class ClusterClient[T](clusterClient: FlinkClusterClient[T]) extends
ClusterClientTrait[T](clusterClient) {
-
- override def cancelWithSavepoint(jobID: JobID, s: String):
CompletableFuture[String] = clusterClient.cancelWithSavepoint(jobID, s)
-
- override def stopWithSavepoint(jobID: JobID, b: Boolean, s: String):
CompletableFuture[String] = clusterClient.stopWithSavepoint(jobID, b, s)
-
- override def triggerSavepoint(jobID: JobID, s: String):
CompletableFuture[String] = clusterClient.triggerSavepoint(jobID, s)
+class FlinkClusterClient[T](clusterClient: ClusterClient[T]) extends
FlinkClientTrait[T](clusterClient) {
}
diff --git
a/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.12/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
b/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.13/src/main/scala/org/apache/streampark/flink/core/FlinkClusterClient.scala
similarity index 55%
copy from
streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.12/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
copy to
streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.13/src/main/scala/org/apache/streampark/flink/core/FlinkClusterClient.scala
index eed165104..6549e5e24 100644
---
a/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.12/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
+++
b/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.13/src/main/scala/org/apache/streampark/flink/core/FlinkClusterClient.scala
@@ -15,17 +15,9 @@
* limitations under the License.
*/
package org.apache.streampark.flink.core
-import org.apache.flink.api.common.JobID
-import org.apache.flink.client.program.{ClusterClient => FlinkClusterClient}
-import java.util.concurrent.CompletableFuture
+import org.apache.flink.client.program.ClusterClient
-class ClusterClient[T](clusterClient: FlinkClusterClient[T]) extends
ClusterClientTrait[T](clusterClient) {
-
- override def cancelWithSavepoint(jobID: JobID, s: String):
CompletableFuture[String] = clusterClient.cancelWithSavepoint(jobID, s)
-
- override def stopWithSavepoint(jobID: JobID, b: Boolean, s: String):
CompletableFuture[String] = clusterClient.stopWithSavepoint(jobID, b, s)
-
- override def triggerSavepoint(jobID: JobID, s: String):
CompletableFuture[String] = clusterClient.triggerSavepoint(jobID, s)
+class FlinkClusterClient[T](clusterClient: ClusterClient[T]) extends
FlinkClientTrait[T](clusterClient) {
}
diff --git
a/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.12/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
b/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.14/src/main/scala/org/apache/streampark/flink/core/FlinkClusterClient.scala
similarity index 55%
copy from
streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.12/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
copy to
streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.14/src/main/scala/org/apache/streampark/flink/core/FlinkClusterClient.scala
index eed165104..6549e5e24 100644
---
a/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.12/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
+++
b/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.14/src/main/scala/org/apache/streampark/flink/core/FlinkClusterClient.scala
@@ -15,17 +15,9 @@
* limitations under the License.
*/
package org.apache.streampark.flink.core
-import org.apache.flink.api.common.JobID
-import org.apache.flink.client.program.{ClusterClient => FlinkClusterClient}
-import java.util.concurrent.CompletableFuture
+import org.apache.flink.client.program.ClusterClient
-class ClusterClient[T](clusterClient: FlinkClusterClient[T]) extends
ClusterClientTrait[T](clusterClient) {
-
- override def cancelWithSavepoint(jobID: JobID, s: String):
CompletableFuture[String] = clusterClient.cancelWithSavepoint(jobID, s)
-
- override def stopWithSavepoint(jobID: JobID, b: Boolean, s: String):
CompletableFuture[String] = clusterClient.stopWithSavepoint(jobID, b, s)
-
- override def triggerSavepoint(jobID: JobID, s: String):
CompletableFuture[String] = clusterClient.triggerSavepoint(jobID, s)
+class FlinkClusterClient[T](clusterClient: ClusterClient[T]) extends
FlinkClientTrait[T](clusterClient) {
}
diff --git
a/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.15/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
b/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.15/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
deleted file mode 100644
index f0c655e98..000000000
---
a/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.15/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
+++ /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.streampark.flink.core
-
-import org.apache.flink.api.common.JobID
-import org.apache.flink.client.program.{ClusterClient => FlinkClusterClient}
-import org.apache.flink.core.execution.SavepointFormatType
-
-import java.util.concurrent.CompletableFuture
-import javax.annotation.Nullable
-
-class ClusterClient[T](clusterClient: FlinkClusterClient[T]) extends
ClusterClientTrait[T](clusterClient) {
-
- override def cancelWithSavepoint(jobId: JobID, @Nullable savepointDirectory:
String, formatType: SavepointFormatType): CompletableFuture[String] = {
- clusterClient.cancelWithSavepoint(jobId, savepointDirectory, formatType)
- }
-
- override def stopWithSavepoint(jobId: JobID,
- advanceToEndOfEventTime: Boolean,
- @Nullable savepointDir: String,
- formatType: SavepointFormatType):
CompletableFuture[String] = {
- clusterClient.stopWithSavepoint(jobId, advanceToEndOfEventTime,
savepointDir, formatType)
- }
-
- override def triggerSavepoint(jobId: JobID, @Nullable savepointDir: String,
formatType: SavepointFormatType): CompletableFuture[String] = {
- clusterClient.triggerSavepoint(jobId, savepointDir, formatType)
- }
-
- def cancelWithSavepoint(jobID: JobID, savepointDirectory: String):
CompletableFuture[String] = {
- clusterClient.cancelWithSavepoint(jobID, savepointDirectory,
SavepointFormatType.DEFAULT)
- }
-
- def stopWithSavepoint(jobID: JobID, advanceToEndOfEventTime: Boolean,
savepointDirectory: String): CompletableFuture[String] = {
- clusterClient.stopWithSavepoint(jobID, advanceToEndOfEventTime,
savepointDirectory, SavepointFormatType.DEFAULT)
- }
-
-}
diff --git
a/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.12/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
b/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.15/src/main/scala/org/apache/streampark/flink/core/FlinkClusterClient.scala
similarity index 57%
rename from
streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.12/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
rename to
streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.15/src/main/scala/org/apache/streampark/flink/core/FlinkClusterClient.scala
index eed165104..f7d48dc4d 100644
---
a/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.12/src/main/scala/org/apache/streampark/flink/core/ClusterClient.scala
+++
b/streampark-flink/streampark-flink-shims/streampark-flink-shims_flink-1.15/src/main/scala/org/apache/streampark/flink/core/FlinkClusterClient.scala
@@ -15,17 +15,21 @@
* limitations under the License.
*/
package org.apache.streampark.flink.core
+
import org.apache.flink.api.common.JobID
-import org.apache.flink.client.program.{ClusterClient => FlinkClusterClient}
+import org.apache.flink.client.program.ClusterClient
+import org.apache.flink.core.execution.SavepointFormatType
import java.util.concurrent.CompletableFuture
-class ClusterClient[T](clusterClient: FlinkClusterClient[T]) extends
ClusterClientTrait[T](clusterClient) {
-
- override def cancelWithSavepoint(jobID: JobID, s: String):
CompletableFuture[String] = clusterClient.cancelWithSavepoint(jobID, s)
+class FlinkClusterClient[T](clusterClient: ClusterClient[T]) extends
FlinkClientTrait[T](clusterClient) {
- override def stopWithSavepoint(jobID: JobID, b: Boolean, s: String):
CompletableFuture[String] = clusterClient.stopWithSavepoint(jobID, b, s)
+ override def cancelWithSavepoint(jobID: JobID, savepointDirectory: String):
CompletableFuture[String] = {
+ clusterClient.cancelWithSavepoint(jobID, savepointDirectory,
SavepointFormatType.DEFAULT)
+ }
- override def triggerSavepoint(jobID: JobID, s: String):
CompletableFuture[String] = clusterClient.triggerSavepoint(jobID, s)
+ override def stopWithSavepoint(jobID: JobID, advanceToEndOfEventTime:
Boolean, savepointDirectory: String): CompletableFuture[String] = {
+ clusterClient.stopWithSavepoint(jobID, advanceToEndOfEventTime,
savepointDirectory, SavepointFormatType.DEFAULT)
+ }
}
diff --git
a/streampark-flink/streampark-flink-submit/streampark-flink-submit-core/src/main/scala/org/apache/streampark/flink/submit/trait/FlinkSubmitTrait.scala
b/streampark-flink/streampark-flink-submit/streampark-flink-submit-core/src/main/scala/org/apache/streampark/flink/submit/trait/FlinkSubmitTrait.scala
index bcc26e5f6..93e00d147 100644
---
a/streampark-flink/streampark-flink-submit/streampark-flink-submit-core/src/main/scala/org/apache/streampark/flink/submit/trait/FlinkSubmitTrait.scala
+++
b/streampark-flink/streampark-flink-submit/streampark-flink-submit-core/src/main/scala/org/apache/streampark/flink/submit/trait/FlinkSubmitTrait.scala
@@ -35,12 +35,12 @@ import org.apache.streampark.common.conf.{ConfigConst,
Workspace}
import org.apache.streampark.common.enums.{ApplicationType, DevelopmentMode,
ExecutionMode, ResolveOrder}
import org.apache.streampark.common.util.{Logger, SystemPropertyUtils, Utils}
import org.apache.streampark.flink.core.conf.FlinkRunOption
-import org.apache.streampark.flink.core.{ClusterClient => ClusterClientWrapper}
+import org.apache.streampark.flink.core.FlinkClusterClient
import org.apache.streampark.flink.submit.bean._
+
import java.io.File
import java.util.concurrent.TimeUnit
import java.util.{Collections, List => JavaList, Map => JavaMap}
-
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
import scala.collection.mutable
@@ -486,11 +486,11 @@ trait FlinkSubmitTrait extends Logger {
val clientTimeout =
getOptionFromDefaultFlinkConfig(cancelRequest.flinkVersion.flinkHome,
ClientOptions.CLIENT_TIMEOUT)
- val clientWrapper = new ClusterClientWrapper(client)
+ val clientWrapper = new FlinkClusterClient(client)
(Try(cancelRequest.withSavePoint).getOrElse(false),
Try(cancelRequest.withDrain).getOrElse(false)) match {
case (false, false) =>
- clientWrapper.cancel(jobID).get()
+ client.cancel(jobID).get()
null
case (true, false) => clientWrapper.cancelWithSavepoint(jobID,
savePointDir).get(clientTimeout.toMillis, TimeUnit.MILLISECONDS)
case (_, _) => clientWrapper.stopWithSavepoint(jobID,
cancelRequest.withDrain, savePointDir).get(clientTimeout.toMillis,
TimeUnit.MILLISECONDS)