You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hop.apache.org by mc...@apache.org on 2021/07/29 19:59:54 UTC
[incubator-hop] branch master updated: HOP-3140 : Beam Dataflow:
add option to use public IPs or not
This is an automated email from the ASF dual-hosted git repository.
mcasters pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-hop.git
The following commit(s) were added to refs/heads/master by this push:
new 6ada1c3 HOP-3140 : Beam Dataflow: add option to use public IPs or not
new ebeb690 Merge pull request #975 from mattcasters/master
6ada1c3 is described below
commit 6ada1c362d2d482d1b0e48e6d8c47863bcefeaeb
Author: Matt Casters <ma...@gmail.com>
AuthorDate: Thu Jul 29 21:58:51 2021 +0200
HOP-3140 : Beam Dataflow: add option to use public IPs or not
---
.../beam-dataflow-pipeline-engine.adoc | 6 +++++
.../BeamDataFlowPipelineRunConfiguration.java | 31 ++++++++++++++++++++++
2 files changed, 37 insertions(+)
diff --git a/docs/hop-user-manual/modules/ROOT/pages/pipeline/pipeline-run-configurations/beam-dataflow-pipeline-engine.adoc b/docs/hop-user-manual/modules/ROOT/pages/pipeline/pipeline-run-configurations/beam-dataflow-pipeline-engine.adoc
index b2dc1c7..aee87cd 100644
--- a/docs/hop-user-manual/modules/ROOT/pages/pipeline/pipeline-run-configurations/beam-dataflow-pipeline-engine.adoc
+++ b/docs/hop-user-manual/modules/ROOT/pages/pipeline/pipeline-run-configurations/beam-dataflow-pipeline-engine.adoc
@@ -102,6 +102,12 @@ The default is up to the Dataflow service.
For more information, see the reference documentation https://cloud.google.com/compute/docs/networking.
The default is up to the Dataflow service.
+|Use public IPs?
+|Specifies whether worker pools should be started with public IP addresses.
+*WARNING*: This feature is experimental.
+You must be allowlisted to use it.
+
+
|User agent|A user agent string as per https://tools.ietf.org/html/rfc2616[RFC2616], describing the pipeline to external services.
|Temp location|Cloud Storage path for temporary files.
Must be a valid Cloud Storage URL, beginning with gs://.
diff --git a/plugins/engines/beam/src/main/java/org/apache/hop/beam/engines/dataflow/BeamDataFlowPipelineRunConfiguration.java b/plugins/engines/beam/src/main/java/org/apache/hop/beam/engines/dataflow/BeamDataFlowPipelineRunConfiguration.java
index 5fc3362..c12416e 100644
--- a/plugins/engines/beam/src/main/java/org/apache/hop/beam/engines/dataflow/BeamDataFlowPipelineRunConfiguration.java
+++ b/plugins/engines/beam/src/main/java/org/apache/hop/beam/engines/dataflow/BeamDataFlowPipelineRunConfiguration.java
@@ -154,9 +154,21 @@ public class BeamDataFlowPipelineRunConfiguration extends BeamPipelineRunConfigu
@HopMetadataProperty
private String gcpSubnetwork;
+ @GuiWidgetElement(
+ order = "20130-dataflow-options",
+ parentId = PipelineRunConfiguration.GUI_PLUGIN_ELEMENT_PARENT_ID,
+ type = GuiElementType.CHECKBOX,
+ label = "Use public IPs?",
+ toolTip =
+ "Specifies whether worker pools should be started with public IP addresses."
+ + " WARNING: This feature is experimental. You must be allowlisted to use it.")
+ @HopMetadataProperty
+ private boolean gcpUsingPublicIps;
+
public BeamDataFlowPipelineRunConfiguration() {
super();
this.gcpAppName = "Hop";
+ this.gcpUsingPublicIps = true;
}
// Clone
@@ -175,6 +187,7 @@ public class BeamDataFlowPipelineRunConfiguration extends BeamPipelineRunConfigu
this.gcpZone = config.gcpZone;
this.gcpNetwork = config.gcpNetwork;
this.gcpSubnetwork = config.gcpSubnetwork;
+ this.gcpUsingPublicIps = config.gcpUsingPublicIps;
}
public BeamDataFlowPipelineRunConfiguration clone() {
@@ -249,6 +262,10 @@ public class BeamDataFlowPipelineRunConfiguration extends BeamPipelineRunConfigu
options.setSubnetwork(subnetwork);
}
+ // Experimental feature...
+ //
+ options.setUsePublicIps(isGcpUsingPublicIps());
+
if (StringUtils.isNotEmpty(getFatJar())) {
options.setFilesToStage(Arrays.asList(resolve(fatJar)));
}
@@ -442,4 +459,18 @@ public class BeamDataFlowPipelineRunConfiguration extends BeamPipelineRunConfigu
public void setGcpSubnetwork(String gcpSubnetwork) {
this.gcpSubnetwork = gcpSubnetwork;
}
+
+ /**
+ * Gets gcpUsingPublicIps
+ *
+ * @return value of gcpUsingPublicIps
+ */
+ public boolean isGcpUsingPublicIps() {
+ return gcpUsingPublicIps;
+ }
+
+ /** @param gcpUsingPublicIps The gcpUsingPublicIps to set */
+ public void setGcpUsingPublicIps(boolean gcpUsingPublicIps) {
+ this.gcpUsingPublicIps = gcpUsingPublicIps;
+ }
}