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

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

                Author: ASF GitHub Bot
            Created on: 13/Nov/18 15:38
            Start Date: 13/Nov/18 15:38
    Worklog Time Spent: 10m 
      Work Description: mxm closed pull request #6986: [BEAM-5999] Reconcile 
timer proto representation.
URL: https://github.com/apache/beam/pull/6986
 
 
   

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/model/pipeline/src/main/proto/beam_runner_api.proto 
b/model/pipeline/src/main/proto/beam_runner_api.proto
index 194797f166c..27ab1f1011b 100644
--- a/model/pipeline/src/main/proto/beam_runner_api.proto
+++ b/model/pipeline/src/main/proto/beam_runner_api.proto
@@ -509,6 +509,7 @@ message SetStateSpec {
 
 message TimerSpec {
   TimeDomain.Enum time_domain = 1;
+  string timer_coder_id = 2;
 }
 
 message IsBounded {
diff --git 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
index 04e03d61023..568b4282540 100644
--- 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
+++ 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
@@ -218,7 +218,7 @@ public SdkFunctionSpec translateDoFn(SdkComponents 
newComponents) {
             for (Map.Entry<String, TimerDeclaration> timer :
                 signature.timerDeclarations().entrySet()) {
               RunnerApi.TimerSpec spec =
-                  translateTimerSpec(getTimerSpecOrThrow(timer.getValue(), 
doFn));
+                  translateTimerSpec(getTimerSpecOrThrow(timer.getValue(), 
doFn), newComponents);
               timerSpecs.put(timer.getKey(), spec);
             }
             return timerSpecs;
@@ -479,9 +479,12 @@ private static String registerCoderOrThrow(SdkComponents 
components, Coder coder
     }
   }
 
-  public static RunnerApi.TimerSpec translateTimerSpec(TimerSpec timer) {
+  public static RunnerApi.TimerSpec translateTimerSpec(TimerSpec timer, 
SdkComponents components) {
     return RunnerApi.TimerSpec.newBuilder()
         .setTimeDomain(translateTimeDomain(timer.getTimeDomain()))
+        // TODO: Add support for timer payloads to the SDK
+        // We currently assume that all payloads are unspecified.
+        .setTimerCoderId(registerCoderOrThrow(components, 
Timer.Coder.of(VoidCoder.of())))
         .build();
   }
 
diff --git 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java
 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java
index 5e36307438f..ec959ea3a8d 100644
--- 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java
+++ 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java
@@ -364,8 +364,6 @@ private static ExecutableStage 
sanitizeDanglingPTransformInputs(ExecutableStage
             .stream()
             .map(PCollectionNode::getId)
             .collect(Collectors.toSet()));
-    possibleInputs.addAll(
-        stage.getTimers().stream().map(t -> 
t.collection().getId()).collect(Collectors.toSet()));
     possibleInputs.addAll(
         stage
             .getSideInputs()
diff --git 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
index 280e705db6c..bb6a47b7555 100644
--- 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
+++ 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
@@ -382,12 +382,8 @@ public Components getComponents() {
             localName -> {
               String transformId = transform.getId();
               PTransform transformProto = 
components.getTransformsOrThrow(transformId);
-              String collectionId = 
transform.getTransform().getInputsOrThrow(localName);
-              PCollection collection = 
components.getPcollectionsOrThrow(collectionId);
               return TimerReference.of(
-                  PipelineNode.pTransform(transformId, transformProto),
-                  localName,
-                  PipelineNode.pCollection(collectionId, collection));
+                  PipelineNode.pTransform(transformId, transformProto), 
localName);
             })
         .collect(Collectors.toSet());
   }
diff --git 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/TimerReference.java
 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/TimerReference.java
index 0c6bf3a36bc..ac864c8644b 100644
--- 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/TimerReference.java
+++ 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/TimerReference.java
@@ -27,12 +27,10 @@
  */
 @AutoValue
 public abstract class TimerReference {
+
   /** Create a timer reference. */
-  public static TimerReference of(
-      PipelineNode.PTransformNode transform,
-      String localName,
-      PipelineNode.PCollectionNode collection) {
-    return new AutoValue_TimerReference(transform, localName, collection);
+  public static TimerReference of(PipelineNode.PTransformNode transform, 
String localName) {
+    return new AutoValue_TimerReference(transform, localName);
   }
 
   /** Create a timer reference from a TimerId proto and components. */
@@ -40,19 +38,12 @@ public static TimerReference fromTimerId(
       RunnerApi.ExecutableStagePayload.TimerId timerId, RunnerApi.Components 
components) {
     String transformId = timerId.getTransformId();
     String localName = timerId.getLocalName();
-    String collectionId = 
components.getTransformsOrThrow(transformId).getInputsOrThrow(localName);
     RunnerApi.PTransform transform = 
components.getTransformsOrThrow(transformId);
-    RunnerApi.PCollection collection = 
components.getPcollectionsOrThrow(collectionId);
-    return of(
-        PipelineNode.pTransform(transformId, transform),
-        localName,
-        PipelineNode.pCollection(collectionId, collection));
+    return of(PipelineNode.pTransform(transformId, transform), localName);
   }
 
   /** The PTransform that uses this timer. */
   public abstract PipelineNode.PTransformNode transform();
   /** The local name the referencing PTransform uses to refer to this timer. */
   public abstract String localName();
-  /** The PCollection that backs this timer. */
-  public abstract PipelineNode.PCollectionNode collection();
 }
diff --git 
a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java
 
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java
index a88e45e9ddd..ea10a52d321 100644
--- 
a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java
+++ 
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java
@@ -94,8 +94,7 @@ public void testRoundTripToFromTransform() throws Exception {
     UserStateReference userStateRef =
         UserStateReference.of(
             transformNode, "user_state", PipelineNode.pCollection("input.out", 
input));
-    TimerReference timerRef =
-        TimerReference.of(transformNode, "timer", 
PipelineNode.pCollection("timer.out", timer));
+    TimerReference timerRef = TimerReference.of(transformNode, "timer");
     ImmutableExecutableStage stage =
         ImmutableExecutableStage.of(
             components,
diff --git 
a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java
 
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java
index 3c7c5b20bc7..fd034d8ac3f 100644
--- 
a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java
+++ 
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java
@@ -88,8 +88,7 @@ public void ofFullComponentsOnlyHasStagePTransforms() throws 
Exception {
     UserStateReference userStateRef =
         UserStateReference.of(
             transformNode, "user_state", PipelineNode.pCollection("input.out", 
input));
-    TimerReference timerRef =
-        TimerReference.of(transformNode, "timer", 
PipelineNode.pCollection("timer.pc", timer));
+    TimerReference timerRef = TimerReference.of(transformNode, "timer");
     ImmutableExecutableStage stage =
         ImmutableExecutableStage.ofFullComponents(
             components,
diff --git 
a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java
 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java
index c82fa091f38..a5bfe61d3aa 100644
--- 
a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java
+++ 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java
@@ -201,7 +201,7 @@ public String getUrn(ParDoSingle<?, ?> transform) {
               for (Map.Entry<String, DoFnSignature.TimerDeclaration> timer :
                   signature.timerDeclarations().entrySet()) {
                 RunnerApi.TimerSpec spec =
-                    translateTimerSpec(getTimerSpecOrThrow(timer.getValue(), 
doFn));
+                    translateTimerSpec(getTimerSpecOrThrow(timer.getValue(), 
doFn), newComponents);
                 timerSpecs.put(timer.getKey(), spec);
               }
               return timerSpecs;
diff --git 
a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
index 39f42d5a014..ee8404654cd 100644
--- 
a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
+++ 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
@@ -24,6 +24,7 @@
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableTable;
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
@@ -40,6 +41,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
+import org.apache.beam.runners.core.construction.ModelCoders;
 import org.apache.beam.runners.core.construction.SyntheticComponents;
 import org.apache.beam.runners.core.construction.graph.ExecutableStage;
 import org.apache.beam.runners.core.construction.graph.PipelineNode;
@@ -333,22 +335,56 @@ private static TargetEncoding addStageOutput(
           throw new IllegalArgumentException(String.format("Unknown time 
domain %s", timeDomain));
       }
 
+      String mainInputName =
+          timerReference
+              .transform()
+              .getTransform()
+              .getInputsOrThrow(
+                  Iterables.getOnlyElement(
+                      Sets.difference(
+                          
timerReference.transform().getTransform().getInputsMap().keySet(),
+                          Sets.union(
+                              payload.getSideInputsMap().keySet(),
+                              payload.getTimerSpecsMap().keySet()))));
+      String timerCoderId =
+          keyValueCoderId(
+              components
+                  
.getCodersOrThrow(components.getPcollectionsOrThrow(mainInputName).getCoderId())
+                  .getComponentCoderIds(0),
+              
payload.getTimerSpecsOrThrow(timerReference.localName()).getTimerCoderId(),
+              components);
+      RunnerApi.PCollection timerCollectionSpec =
+          components
+              .getPcollectionsOrThrow(mainInputName)
+              .toBuilder()
+              .setCoderId(timerCoderId)
+              .build();
+
+      // "Unroll" the timers into PCollections.
+      String inputTimerPCollectionId =
+          SyntheticComponents.uniqueId(
+              String.format(
+                  "%s.timer.%s.in", timerReference.transform().getId(), 
timerReference.localName()),
+              components.getPcollectionsMap()::containsKey);
+      components.putPcollections(inputTimerPCollectionId, timerCollectionSpec);
       remoteInputsBuilder.put(
-          timerReference.collection().getId(),
-          addStageInput(dataEndpoint, timerReference.collection(), 
components));
-      // "Unroll" the timer PCollection to make the execution tree a DAG.
+          inputTimerPCollectionId,
+          addStageInput(
+              dataEndpoint,
+              PipelineNode.pCollection(inputTimerPCollectionId, 
timerCollectionSpec),
+              components));
       String outputTimerPCollectionId =
           SyntheticComponents.uniqueId(
-              String.format("%s.out", timerReference.collection().getId()),
+              String.format(
+                  "%s.timer.%s.out",
+                  timerReference.transform().getId(), 
timerReference.localName()),
               components.getPcollectionsMap()::containsKey);
-      components.putPcollections(
-          outputTimerPCollectionId, 
timerReference.collection().getPCollection());
+      components.putPcollections(outputTimerPCollectionId, 
timerCollectionSpec);
       TargetEncoding targetEncoding =
           addStageOutput(
               dataEndpoint,
               components,
-              PipelineNode.pCollection(
-                  outputTimerPCollectionId, 
timerReference.collection().getPCollection()));
+              PipelineNode.pCollection(outputTimerPCollectionId, 
timerCollectionSpec));
       outputTargetCodersBuilder.put(targetEncoding.getTarget(), 
targetEncoding.getCoder());
       components.putTransforms(
           timerReference.transform().getId(),
@@ -356,6 +392,7 @@ private static TargetEncoding addStageOutput(
           components
               .getTransformsOrThrow(timerReference.transform().getId())
               .toBuilder()
+              .putInputs(timerReference.localName(), inputTimerPCollectionId)
               .putOutputs(timerReference.localName(), outputTimerPCollectionId)
               .build());
 
@@ -365,13 +402,32 @@ private static TargetEncoding addStageOutput(
           TimerSpec.of(
               timerReference.transform().getId(),
               timerReference.localName(),
-              timerReference.collection().getId(),
+              inputTimerPCollectionId,
               targetEncoding.getTarget(),
               spec));
     }
     return idsToSpec.build().rowMap();
   }
 
+  private static String keyValueCoderId(
+      String keyCoderId, String valueCoderId, Components.Builder components) {
+    String id =
+        uniqueId(
+            String.format("kv-%s-%s", keyCoderId, valueCoderId),
+            components.getCodersMap()::containsKey);
+    RunnerApi.Coder.Builder coder;
+    components.putCoders(
+        id,
+        RunnerApi.Coder.newBuilder()
+            .setSpec(
+                RunnerApi.SdkFunctionSpec.newBuilder()
+                    
.setSpec(RunnerApi.FunctionSpec.newBuilder().setUrn(ModelCoders.KV_CODER_URN)))
+            .addComponentCoderIds(keyCoderId)
+            .addComponentCoderIds(valueCoderId)
+            .build());
+    return id;
+  }
+
   @AutoValue
   abstract static class TargetEncoding {
     abstract BeamFnApi.Target getTarget();
diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner.py 
b/sdks/python/apache_beam/runners/portability/fn_api_runner.py
index 832cba95180..526115235e8 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner.py
@@ -862,20 +862,12 @@ def inject_timer_pcollections(stages):
           if transform.spec.urn == common_urns.primitives.PAR_DO.urn:
             payload = proto_utils.parse_Bytes(
                 transform.spec.payload, beam_runner_api_pb2.ParDoPayload)
-            for tag in payload.timer_specs.keys():
+            for tag, spec in payload.timer_specs.items():
               if len(transform.inputs) > 1:
                 raise NotImplementedError('Timers and side inputs.')
               input_pcoll = pipeline_components.pcollections[
                   next(iter(transform.inputs.values()))]
               # Create the appropriate coder for the timer PCollection.
-              void_coder_id = add_or_get_coder_id(
-                  beam.coders.SingletonCoder(None).to_runner_api(None))
-              timer_coder_id = add_or_get_coder_id(
-                  beam_runner_api_pb2.Coder(
-                      spec=beam_runner_api_pb2.SdkFunctionSpec(
-                          spec=beam_runner_api_pb2.FunctionSpec(
-                              urn=common_urns.coders.TIMER.urn)),
-                      component_coder_ids=[void_coder_id]))
               key_coder_id = input_pcoll.coder_id
               if (pipeline_components.coders[key_coder_id].spec.spec.urn
                   == common_urns.coders.WINDOWED_VALUE.urn):
@@ -890,7 +882,7 @@ def inject_timer_pcollections(stages):
                       spec=beam_runner_api_pb2.SdkFunctionSpec(
                           spec=beam_runner_api_pb2.FunctionSpec(
                               urn=common_urns.coders.KV.urn)),
-                      component_coder_ids=[key_coder_id, timer_coder_id]))
+                      component_coder_ids=[key_coder_id, spec.timer_coder_id]))
               timer_pcoll_coder_id = windowed_coder_id(
                   key_timer_coder_id,
                   pipeline_components.windowing_strategies[
diff --git a/sdks/python/apache_beam/transforms/userstate.py 
b/sdks/python/apache_beam/transforms/userstate.py
index c7fc96b1364..0b4f1b9f9e0 100644
--- a/sdks/python/apache_beam/transforms/userstate.py
+++ b/sdks/python/apache_beam/transforms/userstate.py
@@ -26,6 +26,7 @@
 import types
 from builtins import object
 
+from apache_beam.coders import coders
 from apache_beam.coders import Coder
 from apache_beam.portability.api import beam_runner_api_pb2
 from apache_beam.transforms.timeutil import TimeDomain
@@ -95,7 +96,9 @@ def __repr__(self):
 
   def to_runner_api(self, context):
     return beam_runner_api_pb2.TimerSpec(
-        time_domain=TimeDomain.to_runner_api(self.time_domain))
+        time_domain=TimeDomain.to_runner_api(self.time_domain),
+        timer_coder_id=context.coders.get_id(
+            coders._TimerCoder(coders.SingletonCoder(None))))
 
 
 def on_timer(timer_spec):


 

----------------------------------------------------------------
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: 165480)
    Time Spent: 3h 10m  (was: 3h)

> Proto error when running test_pardo_timers of Python PortableValidatesRunner
> ----------------------------------------------------------------------------
>
>                 Key: BEAM-5999
>                 URL: https://issues.apache.org/jira/browse/BEAM-5999
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-py-core, testing
>            Reporter: Maximilian Michels
>            Priority: Major
>              Labels: portability, portability-flink
>             Fix For: 2.9.0
>
>          Time Spent: 3h 10m
>  Remaining Estimate: 0h
>
> When I run {{FlinkRunnerTest.test_pardo_timers}}, I get the following error 
> during fusing the Proto pipeline. Note, the Java ValidatesPortableRunner 
> timer tests run fine. Seems to be an issue with the Python translation.
> {noformat}
> [flink-runner-job-server] ERROR 
> org.apache.beam.runners.flink.FlinkJobInvocation - Error during job 
> invocation 
> test_pardo_timers_1541532770.41_45967eac-8089-4a5f-9d2b-7a4a9d01c7f1.
> java.lang.IllegalArgumentException
>       at 
> org.apache.beam.model.pipeline.v1.RunnerApi$PTransform.getInputsOrThrow(RunnerApi.java:7418)
>       at 
> org.apache.beam.runners.core.construction.graph.QueryablePipeline.lambda$getTimers$11(QueryablePipeline.java:385)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at java.util.Iterator.forEachRemaining(Iterator.java:116)
>       at 
> java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801)
>       at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481)
>       at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471)
>       at 
> java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708)
>       at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>       at 
> java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499)
>       at 
> org.apache.beam.runners.core.construction.graph.QueryablePipeline.getTimers(QueryablePipeline.java:392)
>       at 
> org.apache.beam.runners.core.construction.graph.GreedyStageFuser.forGrpcPortRead(GreedyStageFuser.java:93)
>       at 
> org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser.fuseSiblings(GreedyPipelineFuser.java:340)
>       at 
> org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser.fusePipeline(GreedyPipelineFuser.java:142)
>       at 
> org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser.<init>(GreedyPipelineFuser.java:75)
>       at 
> org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser.fuse(GreedyPipelineFuser.java:88)
>       at 
> org.apache.beam.runners.flink.FlinkJobInvocation.runPipeline(FlinkJobInvocation.java:96)
>       at 
> org.apache.beam.repackaged.beam_runners_flink_2.11.com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:111)
>       at 
> org.apache.beam.repackaged.beam_runners_flink_2.11.com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:58)
>       at 
> org.apache.beam.repackaged.beam_runners_flink_2.11.com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:75)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> {noformat}
> To reproduce, enable the test in {{FlinkRunnerTest}}, then run the following 
> from {{sdks/python}}:
> {noformat}
> python -m apache_beam.runners.portability.flink_runner_test 
> --flink_job_server_jar=/path/to//beam/runners/flink/job-server/build/libs/beam-runners-flink_2.11-job-server-2.9.0-SNAPSHOT.jar
>  --streaming FlinkRunnerTest.test_pardo_timers
> {noformat}



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

Reply via email to