szetszwo commented on code in PR #997:
URL: https://github.com/apache/ratis/pull/997#discussion_r1432964422
##########
ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java:
##########
@@ -279,6 +279,18 @@ static boolean heartbeatChannel(RaftProperties properties)
{
static void setHeartbeatChannel(RaftProperties properties, boolean
useSeparate) {
setBoolean(properties::setBoolean, HEARTBEAT_CHANNEL_KEY, useSeparate);
}
+
+ String EXCEPTIONAL_WARN_LOG_BATCH_DURATION_KEY = PREFIX +
".exceptional-warn-log.batch.duration";
Review Comment:
Let's call it`.log-message.batch.duration`? If we want to have different
durations for warn, error, etc. later on, we could have
- `.log-message.warn.batch.duration`
- `.log-message.error.batch.duration`
##########
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;
Review Comment:
We may remove it by using `startTime != null`.
##########
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;
Review Comment:
Use `int`; see the comment for `increaseCount()`.
##########
ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java:
##########
@@ -204,8 +207,9 @@ private void resetClient(AppendEntriesRequest request,
Event event) {
.map(TermIndex::getIndex)
.orElseGet(f::getMatchIndex);
if (event.isError() && request == null) {
- LOG.warn("{}: Follower failed (request=null, errorCount={}); keep
nextIndex ({}) unchanged and retry.",
- this, errorCount, f.getNextIndex());
+ BatchLogger.warn(LOG, BatchLogger.makeUniqueId(f.getId(),
f.getNextIndex()), () ->
+ LOG.warn("{}: Follower failed (request=null, errorCount={}); keep
nextIndex ({}) unchanged and retry.",
+ this, errorCount, f.getNextIndex()),
exceptionalWarnLogBatchDuration);
Review Comment:
Store `f.getNextIndex()`; otherwise, it may change.
```java
final long followerNextIndex = f.getNextIndex();
BatchLogger.warn(LOG, BatchLogger.makeUniqueId(f.getId(),
followerNextIndex), () ->
LOG.warn("{}: Follower failed (request=null, errorCount={});
keep nextIndex ({}) unchanged and retry.",
this, errorCount, followerNextIndex),
exceptionalWarnLogBatchDuration);
```
##########
ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java:
##########
@@ -279,6 +279,18 @@ static boolean heartbeatChannel(RaftProperties properties)
{
static void setHeartbeatChannel(RaftProperties properties, boolean
useSeparate) {
setBoolean(properties::setBoolean, HEARTBEAT_CHANNEL_KEY, useSeparate);
}
+
+ String EXCEPTIONAL_WARN_LOG_BATCH_DURATION_KEY = PREFIX +
".exceptional-warn-log.batch.duration";
+ TimeDuration EXCEPTIONAL_WARN_LOG_BATCH_DURATION_DEFAULT =
TimeDuration.ZERO; // do not batch
Review Comment:
Let's make the default `5s`?
##########
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() {
+ if (batchStarted.compareAndSet(false, true)) {
+ count.set(0);
+ startTime = Timestamp.currentTime();
+ return true;
+ }
+ return false;
+ }
+ }
+
+ private static final TimeoutExecutor SCHEDULER =
TimeoutExecutor.getInstance();
+ private static final Cache<UniqueId, BatchedLogEntry> LOG_CACHE =
Review Comment:
Use `ConcurrentMap` instead.
```java
//warn(..)
final BatchedLogEntry entry = LOG_CACHE.computeIfAbsent(id, key -> new
BatchedLogEntry(log::warn, op));
if (entry.tryStartBatch()) {
// print the first warn log on batch start
op.run();
SCHEDULER.onTimeout(batchDuration,
() ->
Optional.ofNullable(LOG_CACHE.remove(id)).ifPresent(BatchedLogEntry::execute),
log, () -> "print batched exception failed on " + op);
}
```
##########
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();
+ }
Review Comment:
Combine this method with `tryStartBatch()`. Then, everything becomes
`synchronized`. We can use `int` for count.
##########
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() {
Review Comment:
Print only if `count > 1`, i.e. add
```java
if (count <= 1) {
return;
}
```
##########
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() {
+ if (batchStarted.compareAndSet(false, true)) {
+ count.set(0);
+ startTime = Timestamp.currentTime();
+ return true;
+ }
+ return false;
Review Comment:
Increment `count` here:
```java
if (startTime == null) {
startTime = Timestamp.currentTime();
count = 1;
return true;
}
count++;
return false;
```
##########
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;
Review Comment:
How about using a `String` and a `StackTraceElement` (which has filename and
line number)?
```java
public final static 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();
}
}
```
##########
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() {
+ if (batchStarted.compareAndSet(false, true)) {
+ count.set(0);
+ startTime = Timestamp.currentTime();
+ return true;
+ }
+ return false;
+ }
+ }
+
+ private static final TimeoutExecutor SCHEDULER =
TimeoutExecutor.getInstance();
+ private static final Cache<UniqueId, BatchedLogEntry> LOG_CACHE =
+ CacheBuilder.newBuilder()
+ .expireAfterAccess(1, TimeUnit.MINUTES)
+ .build();
+
+ 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) {
+ if (!shouldBatch || batchDuration.equals(TimeDuration.ZERO)) {
Review Comment:
Use `batchDuration.isNonPositive()`.
##########
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)) {
Review Comment:
Add `if (obj == this)`.
--
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]