[ 
https://issues.apache.org/jira/browse/BEAM-5239?focusedWorklogId=143167&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-143167
 ]

ASF GitHub Bot logged work on BEAM-5239:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 11/Sep/18 16:02
            Start Date: 11/Sep/18 16:02
    Worklog Time Spent: 10m 
      Work Description: asfgit closed pull request #6278: [BEAM-5239] Enable to 
configure latencyTrackingInterval
URL: https://github.com/apache/beam/pull/6278
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java
 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java
index 1162a7ed95a..bf06d0b6699 100644
--- 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java
+++ 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java
@@ -19,6 +19,7 @@
 
 import com.google.common.base.Splitter;
 import java.util.List;
+import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.java.CollectionEnvironment;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.runtime.state.AbstractStateBackend;
@@ -78,6 +79,8 @@ public static ExecutionEnvironment 
createBatchExecutionEnvironment(
       flinkBatchEnv.getConfig().disableObjectReuse();
     }
 
+    applyLatencyTrackingInterval(flinkBatchEnv.getConfig(), options);
+
     return flinkBatchEnv;
   }
 
@@ -163,6 +166,8 @@ public static StreamExecutionEnvironment 
createStreamExecutionEnvironment(
       }
     }
 
+    applyLatencyTrackingInterval(flinkStreamEnv.getConfig(), options);
+
     // State backend
     final AbstractStateBackend stateBackend = options.getStateBackend();
     if (stateBackend != null) {
@@ -171,4 +176,10 @@ public static StreamExecutionEnvironment 
createStreamExecutionEnvironment(
 
     return flinkStreamEnv;
   }
+
+  private static void applyLatencyTrackingInterval(
+      ExecutionConfig config, FlinkPipelineOptions options) {
+    long latencyTrackingInterval = options.getLatencyTrackingInterval();
+    config.setLatencyTrackingInterval(latencyTrackingInterval);
+  }
 }
diff --git 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
index 52b44401ed2..437d09c4295 100644
--- 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
+++ 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
@@ -167,4 +167,12 @@
   Boolean isShutdownSourcesOnFinalWatermark();
 
   void setShutdownSourcesOnFinalWatermark(Boolean shutdownOnFinalWatermark);
+
+  @Description(
+      "Interval in milliseconds for sending latency tracking marks from the 
sources to the sinks. " 
+          + "Interval value <= 0 disables the feature.")
+  @Default.Long(0)
+  Long getLatencyTrackingInterval();
+
+  void setLatencyTrackingInterval(Long interval);
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 143167)
    Time Spent: 3h 10m  (was: 3h)

> Allow configure latencyTrackingInterval
> ---------------------------------------
>
>                 Key: BEAM-5239
>                 URL: https://issues.apache.org/jira/browse/BEAM-5239
>             Project: Beam
>          Issue Type: New Feature
>          Components: runner-flink
>    Affects Versions: 2.6.0
>            Reporter: Jozef Vilcek
>            Assignee: Jozef Vilcek
>            Priority: Major
>             Fix For: 2.7.0
>
>          Time Spent: 3h 10m
>  Remaining Estimate: 0h
>
> Because of FLINK-10226, we need to be able to set 
> latencyTrackingConfiguration for flink via FlinkPipelineOptions



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to