forked from apache/zookeeper
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
ZOOKEEPER-3448: Introduce MessageTracker to assist debug leader and l…
…eaner connectivity issues. We want to have better insight on the state of the world when learners lost connection with leader, so we need capture more information when that happens. We capture more information through MessageTracker which will record the last few sent and received messages at various protocol stage, and these information will be dumped to log files for further analysis. Author: Michael Han <[email protected]> Author: Michael Han <[email protected]> Reviewers: Enrico Olivelli <[email protected]>, Fangmin Lyu <[email protected]> Closes apache#1007 from hanm/twitter/2765eb0629d2f63f07d112270b582e8e931f734f
- Loading branch information
Showing
9 changed files
with
667 additions
and
2 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
103 changes: 103 additions & 0 deletions
103
zookeeper-server/src/main/java/org/apache/zookeeper/server/util/CircularBuffer.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,103 @@ | ||
/** | ||
* 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 | ||
* <p> | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* <p> | ||
* 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.zookeeper.server.util; | ||
|
||
import java.lang.reflect.Array; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
|
||
/** | ||
* Thread safe FIFO CircularBuffer implementation. | ||
* When the buffer is full write operation overwrites the oldest element. | ||
* | ||
* Fun thing @todo, make this lock free as this is called on every quorum message | ||
*/ | ||
public class CircularBuffer<T> { | ||
|
||
private final T[] buffer; | ||
private final int capacity; | ||
private int oldest; | ||
private AtomicInteger numberOfElements = new AtomicInteger(); | ||
|
||
@SuppressWarnings("unchecked") | ||
public CircularBuffer(Class<T> clazz, int capacity) { | ||
if (capacity <= 0) { | ||
throw new IllegalArgumentException("CircularBuffer capacity should be greater than 0"); | ||
} | ||
this.buffer = (T[]) Array.newInstance(clazz, capacity); | ||
this.capacity = capacity; | ||
} | ||
|
||
/** | ||
* Puts elements in the next available index in the array. | ||
* If the array is full the oldest element is replaced with | ||
* the new value. | ||
* @param element | ||
*/ | ||
public synchronized void write(T element) { | ||
int newSize = numberOfElements.incrementAndGet(); | ||
if (newSize > capacity) { | ||
buffer[oldest] = element; | ||
oldest = ++oldest % capacity; | ||
numberOfElements.decrementAndGet(); | ||
} else { | ||
int index = (oldest + numberOfElements.get() - 1) % capacity; | ||
buffer[index] = element; | ||
} | ||
} | ||
|
||
/** | ||
* Reads from the buffer in a FIFO manner. | ||
* Returns the oldest element in the buffer if the buffer ie not empty | ||
* Returns null if the buffer is empty | ||
* @return | ||
*/ | ||
public synchronized T take() { | ||
int newSize = numberOfElements.decrementAndGet(); | ||
if (newSize < 0) { | ||
numberOfElements.incrementAndGet(); | ||
return null; | ||
} | ||
T polled = buffer[oldest]; | ||
oldest = ++oldest % capacity; | ||
return polled; | ||
} | ||
|
||
public synchronized T peek() { | ||
if (numberOfElements.get() <= 0) { | ||
return null; | ||
} | ||
return buffer[oldest]; | ||
} | ||
|
||
public int size() { | ||
return numberOfElements.get(); | ||
} | ||
|
||
public boolean isEmpty() { | ||
return numberOfElements.get() <= 0; | ||
} | ||
|
||
public boolean isFull() { | ||
return numberOfElements.get() >= capacity; | ||
} | ||
|
||
public synchronized void reset() { | ||
numberOfElements.set(0); | ||
} | ||
} |
Oops, something went wrong.