Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -659,11 +659,6 @@ public ConfigurationSource getConfig() {
return config;
}

@VisibleForTesting
public static Logger getLogger() {
return LOG;
}

public void setTimeout(long timeout) {
this.timeout = timeout;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@
* The underlying RPC mechanism can be chosen via the constructor.
*/
public final class XceiverClientRatis extends XceiverClientSpi {
public static final Logger LOG = LoggerFactory.getLogger(XceiverClientRatis.class);
private static final Logger LOG = LoggerFactory.getLogger(XceiverClientRatis.class);

private final Pipeline pipeline;
private final RpcType rpcType;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@
* through to the container.
*/
public class BlockDataStreamOutput implements ByteBufferStreamOutput {
public static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(BlockDataStreamOutput.class);

public static final int PUT_BLOCK_REQUEST_LENGTH_MAX = 1 << 20; // 1MB
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@
*/
public class BlockInputStream extends BlockExtendedInputStream {

public static final Logger LOG = LoggerFactory.getLogger(BlockInputStream.class);
private static final Logger LOG = LoggerFactory.getLogger(BlockInputStream.class);

private static final List<Validator> VALIDATORS =
ContainerProtocolCalls.toValidatorList((request, response) -> validate(response));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@
* through to the container.
*/
public class BlockOutputStream extends OutputStream {
public static final Logger LOG =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Won't this be private as well, or is the default access fine?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is used by subclasses.

static final Logger LOG =
LoggerFactory.getLogger(BlockOutputStream.class);
public static final String EXCEPTION_MSG =
"Unexpected Storage Container Exception: ";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@
* wait until a allocated buffer is released.
*/
public class BufferPool {
public static final Logger LOG = LoggerFactory.getLogger(BufferPool.class);
private static final Logger LOG = LoggerFactory.getLogger(BufferPool.class);

private static final BufferPool EMPTY = new BufferPool(0, 0);
private final int bufferSize;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@
import org.apache.hadoop.ozone.common.Checksum;
import org.apache.hadoop.ozone.common.OzoneChecksumException;
import org.apache.ozone.test.GenericTestUtils;
import org.apache.ozone.test.GenericTestUtils.LogCapturer;
import org.apache.ratis.thirdparty.io.grpc.Status;
import org.apache.ratis.thirdparty.io.grpc.StatusException;
import org.junit.jupiter.api.BeforeEach;
Expand Down Expand Up @@ -266,9 +267,8 @@ public void testSeekAndRead() throws Exception {

@Test
public void testRefreshPipelineFunction() throws Exception {
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
.captureLogs(BlockInputStream.LOG);
GenericTestUtils.setLogLevel(BlockInputStream.LOG, Level.DEBUG);
LogCapturer logCapturer = LogCapturer.captureLogs(BlockInputStream.class);
GenericTestUtils.setLogLevel(BlockInputStream.class, Level.DEBUG);
BlockID blockID = new BlockID(new ContainerBlockID(1, 1));
AtomicBoolean isRefreshed = new AtomicBoolean();
createChunkList(5);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ class TestBufferPool {

@BeforeAll
static void init() {
GenericTestUtils.setLogLevel(BufferPool.LOG, Level.DEBUG);
GenericTestUtils.setLogLevel(BufferPool.class, Level.DEBUG);
}

@Test
Expand All @@ -67,7 +67,7 @@ private void assertAllocationBlockedUntilReleased(BufferPool pool, Deque<ChunkBu
// As the pool is full, allocation will need to wait until a buffer is released.
assertFull(pool);

LogCapturer logCapturer = LogCapturer.captureLogs(BufferPool.LOG);
LogCapturer logCapturer = LogCapturer.captureLogs(BufferPool.class);
AtomicReference<ChunkBuffer> allocated = new AtomicReference<>();
AtomicBoolean allocatorStarted = new AtomicBoolean();
Thread allocator = new Thread(() -> {
Expand Down Expand Up @@ -101,7 +101,7 @@ private void assertAllocationBlocked(BufferPool pool) throws Exception {
// As the pool is full, new allocation will be blocked interruptably if no allocated buffer is released.
assertFull(pool);

LogCapturer logCapturer = LogCapturer.captureLogs(BufferPool.LOG);
LogCapturer logCapturer = LogCapturer.captureLogs(BufferPool.class);
AtomicBoolean allocatorStarted = new AtomicBoolean();
AtomicBoolean interrupted = new AtomicBoolean(false);
Thread allocator = new Thread(() -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ public InnerNodeImpl newInnerNode(String name, String location,
// number of descendant leaves under this node
private int numOfLeaves;
// LOGGER
public static final Logger LOG = LoggerFactory.getLogger(InnerNodeImpl.class);
private static final Logger LOG = LoggerFactory.getLogger(InnerNodeImpl.class);

/**
* Construct an InnerNode from its name, network location, parent, level and
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@
* manages traffic in/out of data centers or racks.
*/
public class NetworkTopologyImpl implements NetworkTopology {
public static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(NetworkTopologyImpl.class);

/** The Inner node crate factory. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@
@InterfaceStability.Evolving
public class ReloadingX509KeyManager extends X509ExtendedKeyManager implements CertificateNotification {

public static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(ReloadingX509KeyManager.class);

private final String type;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@
@InterfaceStability.Evolving
public final class ReloadingX509TrustManager implements X509TrustManager, CertificateNotification {

public static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(ReloadingX509TrustManager.class);

static final String RELOAD_ERROR_MESSAGE =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ public final class JsonUtils {
private static final ObjectMapper MAPPER;
private static final ObjectWriter WRITER;
private static final ObjectMapper INDENT_OUTPUT_MAPPER; // New mapper instance
public static final Logger LOG = LoggerFactory.getLogger(JsonUtils.class);
private static final Logger LOG = LoggerFactory.getLogger(JsonUtils.class);

static {
MAPPER = new ObjectMapper()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@
*/
public class StringCodec implements Codec<StringBuilder> {

public static final Logger LOG = LoggerFactory.getLogger(StringCodec.class);
private static final Logger LOG = LoggerFactory.getLogger(StringCodec.class);
public static final StringFormat FORMAT = new StringFormat();

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,8 +37,7 @@
*/
public abstract class BackgroundService {

@VisibleForTesting
public static final Logger LOG =
protected static final Logger LOG =
LoggerFactory.getLogger(BackgroundService.class);

// Executor to launch child tasks
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@
* for supporting RocksDB direct {@link ByteBuffer} APIs.
*/
public class CodecBuffer implements UncheckedAutoCloseable {
public static final Logger LOG = LoggerFactory.getLogger(CodecBuffer.class);
private static final Logger LOG = LoggerFactory.getLogger(CodecBuffer.class);

/** To create {@link CodecBuffer} instances. */
private static class Factory {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@
* This class is not thread safe.
*/
public class Checksum {
public static final Logger LOG = LoggerFactory.getLogger(Checksum.class);
private static final Logger LOG = LoggerFactory.getLogger(Checksum.class);

private static Function<ByteBuffer, ByteString> newMessageDigestFunction(
String algorithm) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
* For CRC32/CRC32C, each checksum takes 4 bytes. Thus each block chunk has 4 MB / 16 KB * 4 B = 1 KB of checksum data.
*/
public class ChecksumCache {
public static final Logger LOG = LoggerFactory.getLogger(ChecksumCache.class);
private static final Logger LOG = LoggerFactory.getLogger(ChecksumCache.class);

private final int bytesPerChecksum;
private final List<ByteString> checksums;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@
* Utilities for buffers.
*/
public final class BufferUtils {
public static final Logger LOG = LoggerFactory.getLogger(BufferUtils.class);
private static final Logger LOG = LoggerFactory.getLogger(BufferUtils.class);

private static final ByteBuffer[] EMPTY_BYTE_BUFFER_ARRAY = {};

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,14 +26,11 @@
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.EnumSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* Test class for {@link ChecksumCache}.
*/
class TestChecksumCache {
public static final Logger LOG = LoggerFactory.getLogger(TestChecksumCache.class);

@ParameterizedTest
@EnumSource(ChecksumType.class)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@
*/
public final class TimeDurationUtil {

public static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(TimeDurationUtil.class);

private TimeDurationUtil() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -163,10 +163,6 @@ public static void main(String[] args) {
}
}

public static Logger getLogger() {
return LOG;
}

@Override
public Void call() throws Exception {
OzoneConfiguration configuration = getOzoneConf();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,8 +89,7 @@
* State Machine Class.
*/
public class DatanodeStateMachine implements Closeable {
@VisibleForTesting
static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(DatanodeStateMachine.class);
private final ExecutorService executorService;
private final ExecutorService pipelineCommandExecutorService;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@
*/
public class DeleteBlocksCommandHandler implements CommandHandler {

public static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(DeleteBlocksCommandHandler.class);

private final ContainerSet containerSet;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,8 +68,7 @@
*/
public class HeartbeatEndpointTask
implements Callable<EndpointStateMachine.EndPointStates> {
public static final Logger LOG =
LoggerFactory.getLogger(HeartbeatEndpointTask.class);
private static final Logger LOG = LoggerFactory.getLogger(HeartbeatEndpointTask.class);
private final EndpointStateMachine rpcEndpoint;
private DatanodeDetailsProto datanodeDetailsProto;
private StateContext context;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,8 +39,7 @@
*/
public class VersionEndpointTask implements
Callable<EndpointStateMachine.EndPointStates> {
public static final Logger LOG = LoggerFactory.getLogger(VersionEndpointTask
.class);
private static final Logger LOG = LoggerFactory.getLogger(VersionEndpointTask.class);
private final EndpointStateMachine rpcEndPoint;
private final ConfigurationSource configuration;
private final OzoneContainer ozoneContainer;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@
*/
public class CapacityVolumeChoosingPolicy implements VolumeChoosingPolicy {

public static final Logger LOG = LoggerFactory.getLogger(
private static final Logger LOG = LoggerFactory.getLogger(
CapacityVolumeChoosingPolicy.class);

// Stores the index of the next volume to be returned.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@
*/
public class RoundRobinVolumeChoosingPolicy implements VolumeChoosingPolicy {

public static final Logger LOG = LoggerFactory.getLogger(
private static final Logger LOG = LoggerFactory.getLogger(
RoundRobinVolumeChoosingPolicy.class);

// Stores the index of the next volume to be returned.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ public class StorageVolumeChecker {

public static final int MAX_VOLUME_FAILURE_TOLERATED_LIMIT = -1;

public static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(StorageVolumeChecker.class);

private AsyncChecker<Boolean, VolumeCheckResult> delegateChecker;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class ThrottledAsyncChecker<K, V> implements AsyncChecker<K, V> {
public static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(ThrottledAsyncChecker.class);

private final Timer timer;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,7 @@
* log4j.appender.inspectorAppender.layout=org.apache.log4j.PatternLayout
*/
public class KeyValueContainerMetadataInspector implements ContainerInspector {
public static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(KeyValueContainerMetadataInspector.class);
public static final Logger REPORT_LOG = LoggerFactory.getLogger(
"ContainerMetadataInspectorReport");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,7 @@
*/
public class KeyValueHandler extends Handler {

public static final Logger LOG = LoggerFactory.getLogger(
private static final Logger LOG = LoggerFactory.getLogger(
KeyValueHandler.class);

private final BlockManager blockManager;
Expand Down Expand Up @@ -1579,10 +1579,6 @@ private ContainerCommandResponseProto checkFaultInjector(ContainerCommandRequest
return null;
}

public static Logger getLogger() {
return LOG;
}

@VisibleForTesting
public static FaultInjector getInjector() {
return injector;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@
*/
public class BlockManagerImpl implements BlockManager {

public static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(BlockManagerImpl.class);

private ConfigurationSource config;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@
* This class is used to get the DataChannel for streaming.
*/
public class KeyValueStreamDataChannel extends StreamDataChannelBase {
public static final Logger LOG =
static final Logger LOG =
LoggerFactory.getLogger(KeyValueStreamDataChannel.class);

interface WriteMethod {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ public class AbstractDatanodeStore extends AbstractRDBStore<AbstractDatanodeDBDe

private Table<String, Long> finalizeBlocksTableWithIterator;

public static final Logger LOG =
protected static final Logger LOG =
LoggerFactory.getLogger(AbstractDatanodeStore.class);

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
* Base class for scheduled scanners on a Datanode.
*/
public abstract class AbstractBackgroundContainerScanner implements Runnable {
public static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(AbstractBackgroundContainerScanner.class);

private final long dataScanInterval;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@
*/
public class BackgroundContainerDataScanner extends
AbstractBackgroundContainerScanner {
public static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(BackgroundContainerDataScanner.class);

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@
*/
public class BackgroundContainerMetadataScanner extends
AbstractBackgroundContainerScanner {
public static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(BackgroundContainerMetadataScanner.class);

private final ContainerMetadataScannerMetrics metrics;
Expand Down
Loading