You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2021/07/14 03:36:00 UTC

[jira] [Work logged] (BEAM-12611) Populate instruction id in Java SDK harness log entries

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

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

                Author: ASF GitHub Bot
            Created on: 14/Jul/21 03:35
            Start Date: 14/Jul/21 03:35
    Worklog Time Spent: 10m 
      Work Description: lukecwik commented on a change in pull request #15166:
URL: https://github.com/apache/beam/pull/15166#discussion_r669249027



##########
File path: sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java
##########
@@ -211,6 +211,11 @@ public void publish(LogRecord record) {
                       .setSeconds(record.getMillis() / 1000)
                       .setNanos((int) (record.getMillis() % 1000) * 1_000_000));
 
+      String instructionId = BeamFnLoggingMDC.getInstructionId();
+      if (!Strings.isNullOrEmpty(instructionId)) {
+        builder.setInstructionId(instructionId);
+      }

Review comment:
       ```suggestion
         String instructionId = BeamFnLoggingMDC.getInstructionId();
         if (instructionId != null) {
           builder.setInstructionId(instructionId);
         }
   ```

##########
File path: sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java
##########
@@ -78,6 +78,7 @@
 

Review comment:
       It is best to write a test rule like https://github.com/apache/beam/blob/master/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/RestoreDataflowLoggingMDC.java and use it in the test instead of relying on everyone doing the appropriate clean-up.

##########
File path: sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingMDC.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.beam.fn.harness.logging;
+
+/** Mapped diagnostic context to be consumed and set on LogEntry protos in BeamFnLoggingClient. */
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)

Review comment:
       +1

##########
File path: sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
##########
@@ -310,6 +311,9 @@ private void createRunnerAndConsumersForPTransformRecursively(
               }
             });
     try {
+      if (!request.getInstructionId().isEmpty()) {
+        BeamFnLoggingMDC.setInstructionId(request.getInstructionId());
+      }

Review comment:
       ```suggestion
         BeamFnLoggingMDC.setInstructionId(request.getInstructionId());
   ```

##########
File path: sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingMDC.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.beam.fn.harness.logging;
+
+/** Mapped diagnostic context to be consumed and set on LogEntry protos in BeamFnLoggingClient. */
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+public class BeamFnLoggingMDC {
+  private static final InheritableThreadLocal<String> instructionId =
+      new InheritableThreadLocal<>();

Review comment:
       ```suggestion
     private static final ThreadLocal<String> instructionId =
         new ThreadLocal<>();
   ```
   
   Inheritable only works if the scope of the child thread is to the instruction that is creating it and if a thread was created in one instruction and is used for multiple instructions then the logged instruction id will be all associated with the original instruction and not any future ones interacting with it.
   
   e.g. `thread-1` for `instruction-A` creates `thread-2`, then `thread-1` finishes `instruction-1` and starts processing `instruction-2` which interacts with `thread-2`, log entries from `thread-1` will be associated with `instruction-1`. Similarly if `thread-3` comes along and interacts with `thread-2` then the logs for `thread-2` will be for `instruction-1`. 
   
   I suggest the `ThreadLocal` since it is easier for people to see that there are logs that aren't associated with an instruction id and for them to learn to manage the `BeamFnLoggingMDC` in child threads then it is for them to find that the wrong instruction id is associated with their logs and to manage the `BeamFnLoggingMDC`.

##########
File path: sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
##########
@@ -362,8 +366,11 @@ private void createRunnerAndConsumersForPTransformRecursively(
       return BeamFnApi.InstructionResponse.newBuilder().setProcessBundle(response);
     } catch (Exception e) {
       // Make sure we clean-up from the active set of bundle processors.
+      BeamFnLoggingMDC.setInstructionId("");
       bundleProcessorCache.discard(bundleProcessor);
       throw e;
+    } finally {
+      BeamFnLoggingMDC.setInstructionId("");

Review comment:
       ```suggestion
         bundleProcessorCache.discard(bundleProcessor);
         throw e;
       } finally {
         BeamFnLoggingMDC.setInstructionId(null);
   ```
   
   The finally is always executed even when an exception is caught.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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

            Worklog Id:     (was: 622357)
    Remaining Estimate: 0h
            Time Spent: 10m

> Populate instruction id in Java SDK harness log entries
> -------------------------------------------------------
>
>                 Key: BEAM-12611
>                 URL: https://issues.apache.org/jira/browse/BEAM-12611
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-java-harness
>            Reporter: Luke Cwik
>            Assignee: Alex Amato
>            Priority: P2
>          Time Spent: 10m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)