szetszwo commented on code in PR #997:
URL: https://github.com/apache/ratis/pull/997#discussion_r1434233581


##########
ratis-common/src/main/java/org/apache/ratis/util/BatchLogger.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.ratis.util;
+
+import org.slf4j.Logger;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Consumer;
+
+/** For batching excessive log messages. */
+public final class BatchLogger {
+
+  private BatchLogger() {
+  }
+
+  public static final class UniqueId {
+    private final StackTraceElement traceElement;

Review Comment:
   Sorry, I recalled that getting a `StackTraceElement` is expensive.  Let's 
add a `Key` interface.  Then, we can keep `UniqueId` private.
   ```java
   //BatchLogger
   +  public interface Key { }
    
   -  public static final class UniqueId {
   +  private static final class UniqueId {
   ```
   
   ```java
   //GrpcLogAppender
     enum BatchLogKey implements BatchLogger.Key {
       RESET_CLIENT,
       APPEND_LOG_RESPONSE_HANDLER_ON_ERROR
     }
   ```
   



##########
ratis-common/src/main/java/org/apache/ratis/util/BatchLogger.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.ratis.util;
+
+import org.apache.ratis.thirdparty.com.google.common.cache.Cache;
+import org.apache.ratis.thirdparty.com.google.common.cache.CacheBuilder;
+import org.apache.ratis.thirdparty.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+
+public final class BatchLogger {
+
+  private BatchLogger() {
+  }
+
+  public final static class UniqueId {
+    private final List<Object> components;
+    private UniqueId(Object... components) {
+      this.components = Lists.newArrayList(components);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (!(obj instanceof UniqueId)) {
+        return false;
+      }
+
+      return components.equals(((UniqueId) obj).components);
+    }
+
+    @Override
+    public int hashCode() {
+      return components.hashCode();
+    }
+  }
+
+  public static UniqueId makeUniqueId(Object... components) {
+    return new UniqueId(components);
+  }
+
+  private final static class BatchedLogEntry {
+    private final Consumer<String> log;
+    private final AtomicInteger count;
+    private final Runnable logOp;
+    private final AtomicBoolean batchStarted;
+    private Timestamp startTime;
+
+    private BatchedLogEntry(Consumer<String> log, Runnable logOp) {
+      this.log = log;
+      this.logOp = logOp;
+      this.count = new AtomicInteger();
+      this.startTime = Timestamp.currentTime();
+      this.batchStarted = new AtomicBoolean(false);
+    }
+
+    private void increaseCount() {
+      this.count.incrementAndGet();
+    }
+
+    private synchronized void execute() {
+      log.accept(String.format("Received %s logs of following between %s and 
%s:",
+          count.get(), startTime, Timestamp.currentTime()));
+      logOp.run();
+      batchStarted.set(false);
+    }
+
+    private synchronized boolean tryStartBatch() {

Review Comment:
   That's is a good point!  Then, we may have dropped some messages in the 
current code in the following step:
   1.  A thread has got an entry from `LOG_CACHE.computeIfAbsent(..)` but has 
not yet called `tryStartBatch(..)`.
   2. The schedule thread removes the entry.
   
   Then, the message in (1) will not be printed.  Let's use `startTime` to 
detect if an entry is already executed; see the other comments.



##########
ratis-common/src/main/java/org/apache/ratis/util/BatchLogger.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.ratis.util;
+
+import org.slf4j.Logger;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Consumer;
+
+/** For batching excessive log messages. */
+public final class BatchLogger {
+
+  private BatchLogger() {
+  }
+
+  public static final class UniqueId {
+    private final StackTraceElement traceElement;
+    private final String name;
+
+    private UniqueId(StackTraceElement traceElement, String name) {
+      this.traceElement = traceElement;
+      this.name = name;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == this) {
+        return true;
+      } else if (!(obj instanceof UniqueId)) {
+        return false;
+      }
+
+      final UniqueId that = (UniqueId) obj;
+      return Objects.equals(this.traceElement, that.traceElement)
+          && Objects.equals(this.name, that.name);
+    }
+
+    @Override
+    public int hashCode() {
+      return traceElement.hashCode() ^ name.hashCode();
+    }
+  }
+
+  public static UniqueId makeUniqueId(String name) {
+    return new UniqueId(JavaUtils.getCallerStackTraceElement(), name);
+  }
+
+  private static final class BatchedLogEntry {
+    private final Consumer<String> log;
+    private final Runnable logOp;
+    private Timestamp startTime = null;
+    private int count = 0;
+
+    private BatchedLogEntry(Consumer<String> log, Runnable logOp) {
+      this.log = log;
+      this.logOp = logOp;
+    }

Review Comment:
   The constructor can be removed and the other methods become:
   ```java
       private synchronized void execute() {
         if (count <= 1) {
           return;
         }
         logOp.accept(String.format(" (Repeated %d times in the last %s)",
             count, startTime.elapsedTime().toString(TimeUnit.SECONDS, 3)));
         startTime = null;
       }
   
       private synchronized boolean tryStartBatch(Consumer<String> op) {
         if (startTime == null) { // already executed
           op.accept("");
           return false;
         }
         logOp = op;
         count++;
         return count == 1; // when count == 1, it is a new entry
       }
   ```



##########
ratis-common/src/main/java/org/apache/ratis/util/BatchLogger.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.ratis.util;
+
+import org.slf4j.Logger;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Consumer;
+
+/** For batching excessive log messages. */
+public final class BatchLogger {
+
+  private BatchLogger() {
+  }
+
+  public static final class UniqueId {
+    private final StackTraceElement traceElement;
+    private final String name;
+
+    private UniqueId(StackTraceElement traceElement, String name) {
+      this.traceElement = traceElement;
+      this.name = name;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == this) {
+        return true;
+      } else if (!(obj instanceof UniqueId)) {
+        return false;
+      }
+
+      final UniqueId that = (UniqueId) obj;
+      return Objects.equals(this.traceElement, that.traceElement)
+          && Objects.equals(this.name, that.name);
+    }
+
+    @Override
+    public int hashCode() {
+      return traceElement.hashCode() ^ name.hashCode();
+    }
+  }
+
+  public static UniqueId makeUniqueId(String name) {
+    return new UniqueId(JavaUtils.getCallerStackTraceElement(), name);
+  }
+
+  private static final class BatchedLogEntry {
+    private final Consumer<String> log;
+    private final Runnable logOp;
+    private Timestamp startTime = null;

Review Comment:
   - We may combine `log` and `logOp`.   `logOp` will take a suffix string 
parameter.
   - Update `logOp` in `tryStartBatch` so it will print the last message.
   - Use `startTime` to determine if the entry is executed.
   - Use `count` to determine if the entry is new.
   ```java
      private static final class BatchedLogEntry {
   -    private final Consumer<String> log;
   -    private final Runnable logOp;
   -    private Timestamp startTime = null;
   +    private Consumer<String> logOp;
   +    private Timestamp startTime = Timestamp.currentTime();
        private int count = 0;
   ```



##########
ratis-common/src/main/java/org/apache/ratis/util/BatchLogger.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.ratis.util;
+
+import org.slf4j.Logger;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Consumer;
+
+/** For batching excessive log messages. */
+public final class BatchLogger {
+
+  private BatchLogger() {
+  }
+
+  public static final class UniqueId {
+    private final StackTraceElement traceElement;
+    private final String name;
+
+    private UniqueId(StackTraceElement traceElement, String name) {
+      this.traceElement = traceElement;
+      this.name = name;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == this) {
+        return true;
+      } else if (!(obj instanceof UniqueId)) {
+        return false;
+      }
+
+      final UniqueId that = (UniqueId) obj;
+      return Objects.equals(this.traceElement, that.traceElement)
+          && Objects.equals(this.name, that.name);
+    }
+
+    @Override
+    public int hashCode() {
+      return traceElement.hashCode() ^ name.hashCode();
+    }
+  }
+
+  public static UniqueId makeUniqueId(String name) {
+    return new UniqueId(JavaUtils.getCallerStackTraceElement(), name);
+  }
+
+  private static final class BatchedLogEntry {
+    private final Consumer<String> log;
+    private final Runnable logOp;
+    private Timestamp startTime = null;
+    private int count = 0;
+
+    private BatchedLogEntry(Consumer<String> log, Runnable logOp) {
+      this.log = log;
+      this.logOp = logOp;
+    }
+
+    private synchronized void execute() {
+      if (count <= 1) {
+        return;
+      }
+      log.accept(String.format("Received %s logs of following in the last %s 
seconds:",
+          count, startTime.elapsedTime()));
+      logOp.run();
+      startTime = null;
+    }
+
+    private synchronized boolean tryStartBatch() {
+      if (startTime == null) {
+        startTime = Timestamp.currentTime();
+        count = 1;
+        return true;
+      }
+      count++;
+      return false;
+    }
+  }
+
+  private static final TimeoutExecutor SCHEDULER = 
TimeoutExecutor.getInstance();
+  private static final ConcurrentMap<UniqueId, BatchedLogEntry> LOG_CACHE = 
new ConcurrentHashMap<>();
+
+  public static void warn(Logger log, UniqueId id, Runnable op, TimeDuration 
batchDuration) {
+    warn(log, id, op, batchDuration, true);
+  }
+
+  public static void warn(Logger log, UniqueId id, Runnable op, TimeDuration 
batchDuration, boolean shouldBatch) {

Review Comment:
   We may expose `Key` and keep `UniqueId` private.
   ```java
     public static void warn(Logger log, Key key, String name, Consumer<String> 
op, TimeDuration batchDuration,
         boolean shouldBatch) {
   ```



-- 
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