wenjin272 commented on code in PR #609:
URL: https://github.com/apache/flink-agents/pull/609#discussion_r3107948831


##########
runtime/src/main/java/org/apache/flink/agents/runtime/eventlog/EventLogRecord.java:
##########
@@ -31,23 +33,82 @@
  *
  * <p>The class uses custom JSON serialization/deserialization to handle 
polymorphic Event types by
  * leveraging the eventType information stored in the EventContext.
+ *
+ * <p>Each record carries a {@link EventLogLevel} that controls truncation 
behavior during
+ * serialization, and a convenience copy of the event type string from the 
context.
  */
 @JsonSerialize(using = EventLogRecordJsonSerializer.class)
 @JsonDeserialize(using = EventLogRecordJsonDeserializer.class)
 public class EventLogRecord {
     private final EventContext context;
     private final Event event;
+    private final EventLogLevel logLevel;
+    private final String eventType;
+    private final transient JsonTruncator truncator;
+    private final transient Counter truncatedEventsCounter;

Review Comment:
   A simple approach might be to move the truncation from the Record 
serialization process into FileEventLogger.append(), like:
   ```
   @Override
   public void append(EventContext context, Event event) throws Exception {
       if (writer == null) {
           throw new IllegalStateException("FileEventLogger not initialized. 
Call open() first.");
       }
   
       String eventTypeStr = EventUtil.resolveEventType(event, context);
   
       // Resolve log level and skip OFF events
       EventLogLevel level =
               levelResolver != null ? levelResolver.resolve(eventTypeStr) : 
EventLogLevel.VERBOSE;
       if (level == EventLogLevel.OFF) {
           return;
       }
   
       EventLogRecord record = new EventLogRecord(context, event, level);
       // All events should be JSON serializable, since we check it when 
sending events to context:
       // RunnerContextImpl.sendEvent
       JsonNode tree = MAPPER.valueToTree(record);
       if (level == EventLogLevel.STANDARD && truncator != null) {
           JsonNode eventNode = tree.get("event");
           if (eventNode instanceof ObjectNode) {
               boolean truncated = truncator.truncate((ObjectNode) eventNode);
               if (truncated && truncatedEventsCounter != null) {
                   truncatedEventsCounter.inc();
               }
           }
       }
       String json =
               prettyPrint
                       ? 
MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(tree)
                       : MAPPER.writeValueAsString(tree);
       writer.println(json);
   }
   ```



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to