From 526e9ac7eca1dd9887b1e913b0c04175347b1660 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 24 Apr 2026 13:36:50 -0700 Subject: [PATCH 1/3] BP-69 base: add slog dependency and LICENSE entries Minimal scaffolding commit for the BP-69 slog migration series: - Add `io.github.merlimat.slog:slog:0.9.7` to the root pom.xml dependencyManagement and to the global compile classpath alongside the existing SLF4J API (which stays as the rendering backend). - Add lombok.config at the repo root so `@CustomLog` generates a slog `Logger` instead of an SLF4J one. - Register the slog jar in LICENSE-all.bin.txt, LICENSE-server.bin.txt and LICENSE-bkctl.bin.txt so the CI check-binary-license script finds it accounted for in the bundled-jars list. No actual Java file is converted in this commit. Individual module migrations stack on top of this one. --- .../src/main/resources/LICENSE-all.bin.txt | 2 ++ .../src/main/resources/LICENSE-bkctl.bin.txt | 2 ++ .../src/main/resources/LICENSE-server.bin.txt | 2 ++ lombok.config | 24 +++++++++++++++++++ pom.xml | 10 ++++++++ 5 files changed, 40 insertions(+) create mode 100644 lombok.config diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt index e23b4fe4b09..868168e9335 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt @@ -216,6 +216,7 @@ Apache Software License, Version 2. - lib/commons-codec-commons-codec-1.18.0.jar [6] - lib/commons-io-commons-io-2.19.0.jar [8] - lib/commons-logging-commons-logging-1.3.5.jar [10] +- lib/io.github.merlimat.slog-slog-0.9.7.jar [64] - lib/io.netty-netty-buffer-4.2.12.Final.jar [11] - lib/io.netty-netty-codec-base-4.2.12.Final.jar [11] - lib/io.netty-netty-codec-compression-4.2.12.Final.jar [11] @@ -422,6 +423,7 @@ Apache Software License, Version 2. [61] Source available at https://github.com/apache/commons-text/tree/rel/commons-text-1.13.1 [62] Source available at https://github.com/apache/commons-beanutils/tree/rel/commons-beanutils-1.11.0 [63] Source available at https://github.com/googleapis/sdk-platform-java/tree/v2.53.0/api-common-java +[64] Source available at https://github.com/merlimat/slog/tree/v0.9.7 ------------------------------------------------------------------------------------ lib/io.netty-netty-codec-base-4.2.12.Final.jar bundles some 3rd party dependencies diff --git a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt index 6982f75955d..dd0d123b0ae 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt @@ -216,6 +216,7 @@ Apache Software License, Version 2. - lib/commons-codec-commons-codec-1.18.0.jar [6] - lib/commons-io-commons-io-2.19.0.jar [8] - lib/commons-logging-commons-logging-1.3.5.jar [10] +- lib/io.github.merlimat.slog-slog-0.9.7.jar [59] - lib/io.netty-netty-buffer-4.2.12.Final.jar [11] - lib/io.netty-netty-codec-base-4.2.12.Final.jar [11] - lib/io.netty-netty-common-4.2.12.Final.jar [11] @@ -355,6 +356,7 @@ Apache Software License, Version 2. [56] Source available at https://github.com/apache/commons-text/tree/rel/commons-text-1.13.1 [57] Source available at https://github.com/apache/commons-beanutils/tree/rel/commons-beanutils-1.11.0 [58] Source available at https://github.com/googleapis/sdk-platform-java/tree/v2.53.0/api-common-java +[59] Source available at https://github.com/merlimat/slog/tree/v0.9.7 ------------------------------------------------------------------------------------ lib/io.netty-netty-codec-base-4.2.12.Final.jar bundles some 3rd party dependencies diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt index b4b7a8f44e0..b2e9d676dc0 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt @@ -216,6 +216,7 @@ Apache Software License, Version 2. - lib/commons-codec-commons-codec-1.18.0.jar [6] - lib/commons-io-commons-io-2.19.0.jar [8] - lib/commons-logging-commons-logging-1.3.5.jar [10] +- lib/io.github.merlimat.slog-slog-0.9.7.jar [63] - lib/io.netty-netty-buffer-4.2.12.Final.jar [11] - lib/io.netty-netty-codec-base-4.2.12.Final.jar [11] - lib/io.netty-netty-codec-compression-4.2.12.Final.jar [11] @@ -417,6 +418,7 @@ Apache Software License, Version 2. [60] Source available at https://github.com/apache/commons-text/tree/rel/commons-text-1.13.1 [61] Source available at https://github.com/apache/commons-beanutils/tree/rel/commons-beanutils-1.11.0 [62] Source available at https://github.com/googleapis/sdk-platform-java/tree/v2.53.0/api-common-java +[63] Source available at https://github.com/merlimat/slog/tree/v0.9.7 ------------------------------------------------------------------------------------ lib/io.netty-netty-codec-base-4.2.12.Final.jar bundles some 3rd party dependencies diff --git a/lombok.config b/lombok.config new file mode 100644 index 00000000000..bc68b26eebd --- /dev/null +++ b/lombok.config @@ -0,0 +1,24 @@ +# +# 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. +# + +# this is the top level Lombok configuration file +# see https://projectlombok.org/features/configuration for reference + +config.stopBubbling = true +lombok.log.custom.declaration = io.github.merlimat.slog.Logger io.github.merlimat.slog.Logger.get(TYPE) diff --git a/pom.xml b/pom.xml index 29d3d24be86..3de84676b52 100644 --- a/pom.xml +++ b/pom.xml @@ -192,6 +192,7 @@ 9.9.3 3.3.0 2.0.12 + 0.9.7 2.0 4.6.0 2.43.0 @@ -273,6 +274,11 @@ + + io.github.merlimat.slog + slog + ${slog.version} + org.slf4j slf4j-bom @@ -882,6 +888,10 @@ + + io.github.merlimat.slog + slog + org.slf4j slf4j-api From 251557c544a87b3cf5242cc44ce120a3abc668f1 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 24 Apr 2026 15:17:23 -0700 Subject: [PATCH 2/3] BP-69: Convert stream module from SLF4J to slog --- .../BKRegistrationNameResolverProvider.java | 15 +- .../clients/SimpleStorageClientImpl.java | 11 +- .../bookkeeper/clients/StorageClientImpl.java | 14 +- .../clients/admin/StorageAdminClientImpl.java | 12 +- .../clients/StorageClientImplTest.java | 4 +- .../channel/StorageServerChannelManager.java | 32 +-- .../container/StorageContainerChannel.java | 18 +- .../StorageContainerClientInterceptor.java | 23 +- .../container/StorageContainerManager.java | 7 +- .../impl/internal/LocationClientImpl.java | 6 +- .../impl/internal/MetaRangeClientImpl.java | 4 +- .../impl/internal/RootRangeClientImpl.java | 4 +- .../RootRangeClientImplWithRetries.java | 6 +- .../StorageServerClientManagerImpl.java | 4 +- .../clients/impl/routing/RangeRouter.java | 4 +- .../clients/utils/GrpcChannels.java | 4 +- .../utils/ListenableFutureRpcProcessor.java | 4 +- .../impl/internal/TestLocationClientImpl.java | 4 +- .../impl/kv/PByteBufSimpleTableImpl.java | 6 +- .../clients/impl/kv/PByteBufTableImpl.java | 14 +- .../impl/kv/PByteBufTableRangeImpl.java | 6 +- .../resolver/ServiceNameResolverProvider.java | 10 +- .../util/AbstractAutoAsyncCloseable.java | 4 +- .../common/router/HashRouterTest.java | 13 +- .../config/ConfigurationSubscription.java | 27 +- .../common/util/SchedulerUtils.java | 6 +- .../apache/distributedlog/io/Abortables.java | 16 +- .../common/config/PropertiesWriter.java | 9 +- .../config/TestConfigurationSubscription.java | 7 +- .../AppendOnlyStreamReader.java | 26 +- .../AppendOnlyStreamWriter.java | 11 +- .../distributedlog/BKAbstractLogWriter.java | 53 ++-- .../distributedlog/BKAsyncLogReader.java | 59 +++-- .../distributedlog/BKAsyncLogWriter.java | 6 +- .../BKDistributedLogManager.java | 29 ++- .../BKDistributedLogNamespace.java | 9 +- .../apache/distributedlog/BKLogHandler.java | 29 ++- .../distributedlog/BKLogReadHandler.java | 21 +- .../distributedlog/BKLogSegmentWriter.java | 120 +++++---- .../distributedlog/BKLogWriteHandler.java | 138 +++++++---- .../distributedlog/BKSyncLogWriter.java | 14 +- .../distributedlog/BookKeeperClient.java | 48 ++-- .../DistributedLogConfiguration.java | 13 +- .../distributedlog/EnvelopedEntryWriter.java | 9 +- .../distributedlog/LocalDLMEmulator.java | 37 +-- .../distributedlog/LogSegmentMetadata.java | 24 +- .../org/apache/distributedlog/MaxTxId.java | 7 +- .../distributedlog/ReadAheadEntryReader.java | 112 +++++---- .../org/apache/distributedlog/ReadUtils.java | 68 ++--- .../distributedlog/ZooKeeperClient.java | 37 +-- .../ZooKeeperClientBuilder.java | 7 +- .../admin/DistributedLogAdmin.java | 22 +- .../api/namespace/NamespaceBuilder.java | 8 +- .../distributedlog/auditor/DLAuditor.java | 79 +++--- .../bk/LedgerAllocatorPool.java | 39 +-- .../distributedlog/bk/QuorumConfig.java | 18 +- .../bk/SimpleLedgerAllocator.java | 84 ++++--- .../config/DynamicConfigurationFactory.java | 7 +- .../feature/AbstractFeatureProvider.java | 8 +- .../feature/ConfigurationFeatureProvider.java | 12 +- .../DynamicConfigurationFeatureProvider.java | 12 +- .../impl/BKNamespaceDriver.java | 78 ++++-- .../impl/ZKLogSegmentFilters.java | 22 +- .../impl/ZKLogSegmentMetadataStore.java | 18 +- .../impl/ZKMetadataAccessor.java | 24 +- .../impl/ZKNamespaceWatcher.java | 20 +- .../impl/acl/ZKAccessControlManager.java | 73 +++--- .../FederatedZKLogMetadataStore.java | 46 ++-- .../logsegment/BKLogSegmentEntryReader.java | 12 +- .../logsegment/BKLogSegmentEntryStore.java | 21 +- .../impl/metadata/BKDLConfig.java | 20 +- .../metadata/ZKLogStreamMetadataStore.java | 60 ++--- .../ZKSubscriptionStateStore.java | 13 +- .../injector/RandomDelayFailureInjector.java | 8 +- .../limiter/ComposableRequestLimiter.java | 5 +- .../distributedlog/lock/LockWaiter.java | 10 +- .../lock/ZKDistributedLock.java | 22 +- .../distributedlog/lock/ZKSessionLock.java | 233 +++++++++++------- .../logsegment/LogSegmentMetadataCache.java | 14 +- .../logsegment/PerStreamLogSegmentCache.java | 37 +-- .../logsegment/TimeBasedRollingPolicy.java | 14 +- .../distributedlog/metadata/DLMetadata.java | 17 +- .../LogSegmentMetadataStoreUpdater.java | 11 +- .../namespace/NamespaceDriverManager.java | 14 +- .../distributedlog/net/DNSResolver.java | 9 +- .../tools/DistributedLogTool.java | 13 +- .../distributedlog/util/FailpointUtils.java | 7 +- .../org/apache/distributedlog/util/Utils.java | 9 +- .../zk/LimitedPermitManager.java | 18 +- .../distributedlog/zk/ZKWatcherManager.java | 27 +- .../apache/distributedlog/DLMTestUtil.java | 16 +- .../NonBlockingReadsTestUtil.java | 21 +- .../TestAppendOnlyStreamReader.java | 6 +- .../TestAppendOnlyStreamWriter.java | 9 +- .../distributedlog/TestAsyncBulkWrite.java | 10 +- .../distributedlog/TestAsyncReaderLock.java | 20 +- .../distributedlog/TestAsyncReaderWriter.java | 76 +++--- .../TestBKDistributedLogManager.java | 42 ++-- .../TestBKDistributedLogNamespace.java | 16 +- .../distributedlog/TestBKLogReadHandler.java | 7 +- .../TestBKLogSegmentWriter.java | 4 +- .../distributedlog/TestBKLogWriteHandler.java | 8 +- .../distributedlog/TestBKSyncLogReader.java | 18 +- .../distributedlog/TestCancelledRead.java | 5 +- .../distributedlog/TestDLMTestUtil.java | 5 +- .../TestDistributedLogBase.java | 19 +- .../TestInterleavedReaders.java | 21 +- .../TestLogSegmentCreation.java | 13 +- .../TestLogSegmentMetadata.java | 10 +- .../distributedlog/TestLogSegmentsZK.java | 5 +- .../distributedlog/TestNonBlockingReads.java | 10 +- .../TestNonBlockingReadsMultiReader.java | 25 +- .../apache/distributedlog/TestReadUtils.java | 7 +- .../org/apache/distributedlog/TestReader.java | 33 ++- .../distributedlog/TestRollLogSegments.java | 47 ++-- .../apache/distributedlog/TestSequenceID.java | 9 +- .../apache/distributedlog/TestTruncate.java | 21 +- .../org/apache/distributedlog/TestTxnId.java | 7 +- .../distributedlog/TestWriteLimiter.java | 6 +- .../distributedlog/TestZooKeeperClient.java | 11 +- .../distributedlog/ZooKeeperClientUtils.java | 10 +- .../ZooKeeperClusterTestCase.java | 8 +- .../acl/TestZKAccessControlManager.java | 26 +- .../apache/distributedlog/admin/TestDLCK.java | 14 +- .../admin/TestDistributedLogAdmin.java | 14 +- .../bk/TestLedgerAllocator.java | 10 +- .../bk/TestLedgerAllocatorPool.java | 16 +- .../config/PropertiesWriter.java | 9 +- .../TestDynamicConfigurationFactory.java | 6 +- .../impl/TestZKLogSegmentFilters.java | 13 +- .../impl/TestZKLogSegmentMetadataStore.java | 22 +- .../TestZKLogStreamMetadataStore.java | 8 +- .../lock/TestDistributedLock.java | 18 +- .../lock/TestZKSessionLock.java | 18 +- .../TestLogSegmentMetadataStoreUpdater.java | 14 +- .../tools/TestDistributedLogTool.java | 7 +- .../distributedlog/fs/DLFileSystem.java | 13 +- .../distributedlog/fs/DLInputStream.java | 4 +- .../distributedlog/fs/DLOutputStream.java | 6 +- .../distributedlog/fs/TestDLFileSystem.java | 4 +- .../EnvelopedRecordSetReader.java | 4 +- .../EnvelopedRecordSetWriter.java | 4 +- .../org/apache/distributedlog/LogRecord.java | 42 ++-- .../distributedlog/TestLogRecordSet.java | 4 +- .../stream/cluster/StandaloneStarter.java | 6 +- .../stream/cluster/StreamCluster.java | 58 +++-- .../stream/server/StorageServer.java | 44 +++- .../stream/server/grpc/GrpcServer.java | 6 +- .../grpc/GrpcStorageContainerService.java | 4 +- .../stream/server/service/BookieService.java | 10 +- .../server/service/BookieWatchService.java | 16 +- .../service/ClusterControllerService.java | 4 +- .../service/CuratorProviderService.java | 8 +- .../service/DLNamespaceProviderService.java | 27 +- .../service/RegistrationServiceProvider.java | 19 +- .../service/RegistrationStateService.java | 10 +- .../stream/server/service/StorageService.java | 4 +- .../bookkeeper/statelib/StateStores.java | 4 +- .../AbstractStateStoreWithJournal.java | 74 +++--- .../statelib/impl/kv/KVCommandProcessor.java | 7 +- .../bookkeeper/statelib/impl/kv/KVUtils.java | 4 +- .../statelib/impl/kv/RocksdbKVStore.java | 25 +- .../impl/mvcc/MVCCAsyncBytesStoreImpl.java | 4 +- .../impl/mvcc/MVCCCommandProcessor.java | 12 +- .../statelib/impl/mvcc/MVCCStoreImpl.java | 4 +- .../statelib/impl/mvcc/MVCCUtils.java | 6 +- .../rocksdb/checkpoint/CheckpointFile.java | 21 +- .../rocksdb/checkpoint/CheckpointInfo.java | 16 +- .../rocksdb/checkpoint/RocksCheckpointer.java | 18 +- .../checkpoint/RocksdbCheckpointTask.java | 18 +- .../checkpoint/RocksdbRestoreTask.java | 10 +- .../checkpoint/dlog/DLCheckpointStore.java | 6 +- .../checkpoint/dlog/DLInputStream.java | 4 +- .../checkpoint/dlog/DLOutputStream.java | 6 +- .../impl/kv/TestRocksdbKVAsyncStore.java | 4 +- ...estRocksdbKVAsyncStoreWithCheckpoints.java | 4 +- .../impl/kv/TestRocksdbKVStoreCheckpoint.java | 4 +- .../statelib/impl/kv/TestStateStore.java | 4 +- .../mvcc/TestMVCCAsyncBytesStoreImpl.java | 10 +- .../statelib/impl/mvcc/TestMVCCStoreImpl.java | 4 +- .../checkpoint/RocksCheckpointerTest.java | 4 +- .../executors/MockExecutorController.java | 6 +- .../cluster/ClusterControllerLeaderImpl.java | 16 +- .../ZkClusterControllerLeaderSelector.java | 4 +- ...usterControllerLeaderSelectorListener.java | 6 +- .../impl/cluster/ZkClusterInitializer.java | 16 +- .../impl/cluster/ZkClusterMetadataStore.java | 8 +- .../impl/grpc/GrpcMetaRangeService.java | 4 +- .../storage/impl/grpc/GrpcTableService.java | 4 +- .../storage/impl/kv/TableStoreImpl.java | 18 +- .../storage/impl/kv/TableStoreUtils.java | 4 +- .../impl/metadata/MetaRangeStoreImpl.java | 4 +- .../impl/metadata/RootRangeStoreImpl.java | 27 +- .../impl/metadata/stream/MetaRangeImpl.java | 12 +- .../impl/routing/RangeRoutingTableImpl.java | 4 +- .../RoutingHeaderProxyInterceptor.java | 36 +-- .../sc/DefaultStorageContainerController.java | 27 +- .../storage/impl/sc/StorageContainerImpl.java | 15 +- .../impl/sc/StorageContainerRegistryImpl.java | 23 +- .../impl/sc/ZkStorageContainerManager.java | 58 +++-- .../impl/service/RangeStoreServiceImpl.java | 4 +- .../impl/store/MVCCStoreFactoryImpl.java | 38 ++- .../impl/TestStorageContainerStoreImpl.java | 4 +- .../ClusterControllerLeaderImplTest.java | 6 +- ...ZkClusterControllerLeaderSelectorTest.java | 8 +- .../cluster/ZkClusterMetadataStoreTest.java | 4 +- .../storage/impl/kv/TableStoreImplTest.java | 4 +- .../impl/metadata/TestRootRangeStoreImpl.java | 4 +- .../metadata/stream/TestMetaRangeImpl.java | 4 +- .../RoutingHeaderProxyInterceptorTest.java | 21 +- ...DefaultStorageContainerControllerTest.java | 10 +- .../impl/store/MVCCStoreFactoryImplTest.java | 11 +- .../bookkeeper/tests/rpc/PingPongService.java | 6 +- 213 files changed, 2151 insertions(+), 1890 deletions(-) diff --git a/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverProvider.java b/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverProvider.java index 1cc1040f178..ed8d0550a20 100644 --- a/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverProvider.java +++ b/stream/bk-grpc-name-resolver/src/main/java/org/apache/bookkeeper/grpc/resolver/BKRegistrationNameResolverProvider.java @@ -24,7 +24,7 @@ import io.grpc.NameResolverProvider; import java.net.URI; import javax.annotation.Nullable; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.net.ServiceURI; import org.apache.bookkeeper.common.resolver.NameResolverFactoryProvider; import org.apache.bookkeeper.common.resolver.NameResolverProviderFactory; @@ -42,7 +42,7 @@ // https://github.com/grpc/grpc-java/issues/7133#issuecomment-680981331 // Skipping the migration for now, will have to deal with this later when GRPC team // finalizes their API. -@Slf4j +@CustomLog public class BKRegistrationNameResolverProvider extends NameResolverFactoryProvider { @Override @@ -63,8 +63,10 @@ public NameResolver newNameResolver(URI targetUri, NameResolver.Args args) { serviceURI = ServiceURI.create(targetUri); } catch (NullPointerException | IllegalArgumentException e) { // invalid uri here, so return null to allow grpc to use other name resolvers - log.info("BKRegistrationNameResolverProvider doesn't know how to resolve {} : cause {}", - targetUri, e.getMessage()); + log.info() + .attr("targetUri", targetUri) + .exceptionMessage(e) + .log("BKRegistrationNameResolverProvider doesn't know how to resolve target URI"); return null; } @@ -73,7 +75,10 @@ public NameResolver newNameResolver(URI targetUri, NameResolver.Args args) { clientDriver = MetadataDrivers.getClientDriver(serviceURI.getUri()); return new BKRegistrationNameResolver(clientDriver, serviceURI.getUri()); } catch (IllegalArgumentException iae) { - log.error("Unknown service uri : {}", serviceURI, iae); + log.error() + .attr("serviceURI", serviceURI) + .exception(iae) + .log("Unknown service uri"); return null; } } diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java index 79ffb0ee2a7..644deb7be8c 100644 --- a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java @@ -28,7 +28,7 @@ import io.netty.buffer.ByteBuf; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.api.StorageClient; import org.apache.bookkeeper.api.exceptions.ApiException; import org.apache.bookkeeper.api.kv.PTable; @@ -50,7 +50,7 @@ /** * The implementation of {@link StorageClient} client. */ -@Slf4j +@CustomLog public class SimpleStorageClientImpl extends SimpleClientBase implements StorageClient { private static final String COMPONENT_NAME = SimpleStorageClientImpl.class.getSimpleName(); @@ -117,8 +117,11 @@ private void openTableImpl(String namespaceName, ).thenCompose(resp -> { if (StatusCode.SUCCESS == resp.getCode()) { StreamProperties streamProps = resp.getStreamProps(); - log.info("Retrieved table properties for table {}/{} : {}", - namespaceName, streamName, streamProps); + log.info() + .attr("namespace", namespaceName) + .attr("streamName", streamName) + .attr("streamProps", streamProps) + .log("Retrieved table properties for table"); if (StorageType.TABLE != streamProps.getStreamConf().getStorageType()) { return FutureUtils.exception(new ApiException( "Can't open a non-table storage entity : " + streamProps.getStreamConf().getStorageType())); diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientImpl.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientImpl.java index dadd4e4f238..2a2d4923f23 100644 --- a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientImpl.java +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/StorageClientImpl.java @@ -22,7 +22,7 @@ import io.netty.buffer.ByteBuf; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.api.StorageClient; import org.apache.bookkeeper.api.exceptions.ApiException; import org.apache.bookkeeper.api.kv.PTable; @@ -44,7 +44,7 @@ /** * The implementation of {@link StorageClient} client. */ -@Slf4j +@CustomLog public class StorageClientImpl extends AbstractAutoAsyncCloseable implements StorageClient { private static final String COMPONENT_NAME = StorageClientImpl.class.getSimpleName(); @@ -122,9 +122,11 @@ private void openTableImpl(String namespaceName, CompletableFuture> future) { FutureUtils.proxyTo( getStreamProperties(namespaceName, tableName).thenComposeAsync(props -> { - if (log.isInfoEnabled()) { - log.info("Retrieved table properties for table {}/{} : {}", namespaceName, tableName, props); - } + log.info() + .attr("namespace", namespaceName) + .attr("tableName", tableName) + .attr("props", props) + .log("Retrieved table properties for table"); if (StorageType.TABLE != props.getStreamConf().getStorageType()) { return FutureUtils.exception(new ApiException( "Can't open a non-table storage entity : " + props.getStreamConf().getStorageType()) @@ -167,7 +169,7 @@ public void close() { try { super.close(1, TimeUnit.SECONDS); } catch (Exception e) { - log.warn("Encountered exceptions on closing the storage client", e); + log.warn().exception(e).log("Encountered exceptions on closing the storage client"); } scheduler.forceShutdown(100, TimeUnit.MILLISECONDS); } diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/StorageAdminClientImpl.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/StorageAdminClientImpl.java index 59638580249..37dd2801fde 100644 --- a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/StorageAdminClientImpl.java +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/StorageAdminClientImpl.java @@ -21,7 +21,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.function.Supplier; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.api.StorageClient; import org.apache.bookkeeper.clients.StorageClientImpl; import org.apache.bookkeeper.clients.config.StorageClientSettings; @@ -38,7 +38,7 @@ /** * A storage admin client. */ -@Slf4j +@CustomLog public class StorageAdminClientImpl extends AbstractAutoAsyncCloseable implements StorageAdminClient { // clients @@ -127,13 +127,9 @@ public void close() { try { closeAsync().get(); } catch (InterruptedException e) { - if (log.isDebugEnabled()) { - log.debug("Interrupted on closing stream admin client", e); - } + log.debug().exception(e).log("Interrupted on closing stream admin client"); } catch (ExecutionException e) { - if (log.isDebugEnabled()) { - log.debug("Failed to cloe stream admin client", e); - } + log.debug().exception(e).log("Failed to cloe stream admin client"); } } } diff --git a/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/StorageClientImplTest.java b/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/StorageClientImplTest.java index ec935cf449a..72eaa870dbc 100644 --- a/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/StorageClientImplTest.java +++ b/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/StorageClientImplTest.java @@ -31,7 +31,7 @@ import static org.mockito.Mockito.when; import io.netty.buffer.ByteBuf; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.api.exceptions.ApiException; import org.apache.bookkeeper.api.kv.PTable; import org.apache.bookkeeper.api.kv.Table; @@ -51,7 +51,7 @@ * Unit test {@link StorageClientImpl}. */ @RunWith(MockitoJUnitRunner.class) -@Slf4j +@CustomLog public class StorageClientImplTest extends GrpcClientTestBase { private static final String NAMESPACE = "test-namespace"; diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/channel/StorageServerChannelManager.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/channel/StorageServerChannelManager.java index 6f035ecf611..ad8b27ff1bb 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/channel/StorageServerChannelManager.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/channel/StorageServerChannelManager.java @@ -24,14 +24,14 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; import javax.annotation.Nullable; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.bookkeeper.stream.proto.common.Endpoint; /** * A manager manages channels to range servers. */ -@Slf4j +@CustomLog public class StorageServerChannelManager implements AutoCloseable { private final ReentrantReadWriteLock lock; @@ -69,21 +69,21 @@ public boolean addStorageServer(Endpoint endpoint, StorageServerChannel channel) lock.readLock().lock(); try { if (closed) { - log.warn("Skip adding channel {} of range server {} since the channel manager is already closed", - channel, endpoint); + log.warn() + .attr("channel", channel) + .attr("endpoint", endpoint) + .log("Skip adding channel since the channel manager is already closed"); channel.close(); return false; } StorageServerChannel oldChannel = channels.putIfAbsent(endpoint, channel); if (null != oldChannel) { - if (log.isDebugEnabled()) { - log.debug("KeyRange server ({}) already existed in the channel manager.", endpoint); - } + log.debug().attr("endpoint", endpoint).log("KeyRange server already existed in the channel manager."); channel.close(); return false; } else { - log.info("Added range server ({}) into the channel manager.", endpoint); + log.info().attr("endpoint", endpoint).log("Added range server into the channel manager."); return true; } } finally { @@ -117,8 +117,10 @@ public StorageServerChannel removeChannel(Endpoint endpoint, StorageServerChanne lock.readLock().lock(); try { if (closed) { - log.warn("Skip removing channel {} of range server {} since the channel manager is already closed", - channel, endpoint); + log.warn() + .attr("channel", channel) + .attr("endpoint", endpoint) + .log("Skip removing channel since the channel manager is already closed"); return null; } @@ -133,12 +135,12 @@ public StorageServerChannel removeChannel(Endpoint endpoint, StorageServerChanne } } if (null == channelRemoved) { - if (log.isDebugEnabled()) { - log.debug("No channel associated with endpoint {} to be removed.", endpoint); - } + log.debug().attr("endpoint", endpoint).log("No channel associated with endpoint to be removed."); } else { - log.info("Removed channel {} for range server {} successfully", - channelRemoved, endpoint); + log.info() + .attr("channel", channelRemoved) + .attr("endpoint", endpoint) + .log("Removed channel for range server successfully"); } if (null != channelRemoved) { channelRemoved.close(); diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerChannel.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerChannel.java index a20d7c9bd44..f5ff7e1e5a4 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerChannel.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerChannel.java @@ -25,7 +25,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import javax.annotation.concurrent.GuardedBy; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.exceptions.StorageContainerException; import org.apache.bookkeeper.clients.impl.channel.StorageServerChannel; import org.apache.bookkeeper.clients.impl.channel.StorageServerChannelManager; @@ -43,7 +43,7 @@ /** * A client place holder for managing information of storage containers. */ -@Slf4j +@CustomLog public class StorageContainerChannel { private final long scId; @@ -139,8 +139,11 @@ private void fetchStorageContainerInfo() { } private void handleFetchStorageContainerInfoFailure(Throwable cause) { - log.info("Failed to fetch info of storage container ({}) - '{}'. Retry in {} ms ...", - new Object[]{scId, cause.getMessage(), ClientConstants.DEFAULT_BACKOFF_START_MS}); + log.info() + .attr("scId", scId) + .exceptionMessage(cause) + .attr("retryMs", ClientConstants.DEFAULT_BACKOFF_START_MS) + .log("Failed to fetch info of storage container. Retrying ..."); executor.schedule(() -> { fetchStorageContainerInfo(); }, ClientConstants.DEFAULT_BACKOFF_START_MS, TimeUnit.MILLISECONDS); @@ -182,8 +185,11 @@ private void handleFetchStorageContainerInfoSuccess( // get the channel from channel manager (if it doesn't exist create one) StorageServerChannel serverChannel = channelManager.getOrCreateChannel(endpoint.getRwEndpoint()); if (null == serverChannel) { - log.info("No channel found/created for range server {}. The channel manager must be shutting down." - + " Stop the process of fetching storage container ({}).", endpoint.getRwEndpoint(), scId); + log.info() + .attr("rwEndpoint", endpoint.getRwEndpoint()) + .attr("scId", scId) + .log("No channel found/created for range server. The channel manager must be shutting down." + + " Stop the process of fetching storage container."); synchronized (this) { rsChannelFuture.completeExceptionally( new ObjectClosedException("StorageServerChannelManager is closed")); diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerClientInterceptor.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerClientInterceptor.java index efdb47f1f22..13479a0bc75 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerClientInterceptor.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerClientInterceptor.java @@ -27,13 +27,13 @@ import io.grpc.ClientInterceptors.CheckedForwardingClientCall; import io.grpc.Metadata; import io.grpc.MethodDescriptor; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.grpc.netty.LongBinaryMarshaller; /** * A client interceptor that intercepting outgoing calls to storage containers. */ -@Slf4j +@CustomLog public class StorageContainerClientInterceptor implements ClientInterceptor { private final long scId; @@ -50,28 +50,23 @@ public StorageContainerClientInterceptor(long scId) { public ClientCall interceptCall(MethodDescriptor method, CallOptions callOptions, Channel next) { - if (log.isTraceEnabled()) { - log.trace("Intercepting method {} : req marshaller = {}, resp marshaller = {}", - method.getFullMethodName(), - method.getRequestMarshaller(), - method.getResponseMarshaller()); - } + log.trace() + .attr("method", method.getFullMethodName()) + .attr("reqMarshaller", method.getRequestMarshaller()) + .attr("respMarshaller", method.getResponseMarshaller()) + .log("Intercepting method"); return new CheckedForwardingClientCall(next.newCall(method, callOptions)) { @Override protected void checkedStart(Listener responseListener, Metadata headers) throws Exception { - if (log.isTraceEnabled()) { - log.trace("Attaching storage container {},", scId); - } + log.trace().attr("scId", scId).log("Attaching storage container"); headers.put(scIdKey, scId); delegate().start(responseListener, headers); } @Override public void request(int numMessages) { - if (log.isTraceEnabled()) { - log.trace("request {} messages", numMessages); - } + log.trace().attr("numMessages", numMessages).log("request messages"); super.request(numMessages); } }; diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerManager.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerManager.java index 35b25b34268..ee04cf0c07e 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerManager.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerManager.java @@ -22,12 +22,12 @@ import java.util.Map; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; /** * A manager manages the mapping between storage containers and range servers. */ -@Slf4j +@CustomLog public class StorageContainerManager implements AutoCloseable { @GuardedBy("storageContainers") @@ -48,7 +48,8 @@ public boolean replaceStorageContainer(long groupId, StorageContainerInfo groupI synchronized (storageContainers) { StorageContainerInfo oldGroupInfo = storageContainers.get(groupId); if (null == oldGroupInfo || oldGroupInfo.getRevision() < groupInfo.getRevision()) { - log.info("Updated the storage container info for group {} : {}", groupId, groupInfo); + log.info().attr("groupId", groupId).attr("groupInfo", groupInfo) + .log("Updated the storage container info"); storageContainers.put(groupId, groupInfo); return true; } diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/LocationClientImpl.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/LocationClientImpl.java index fd46d4b6bcb..e92b5718f06 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/LocationClientImpl.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/LocationClientImpl.java @@ -32,7 +32,7 @@ import java.util.concurrent.CompletableFuture; import java.util.function.Predicate; import java.util.stream.Stream; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.bookkeeper.clients.impl.internal.api.LocationClient; import org.apache.bookkeeper.clients.utils.ClientConstants; @@ -50,7 +50,7 @@ /** * Default Implementation of {@link LocationClient}. */ -@Slf4j +@CustomLog public class LocationClientImpl implements LocationClient { private final StorageClientSettings settings; @@ -83,7 +83,7 @@ private Stream getDefaultBackoffs() { cause -> shouldRetryOnException(cause); private static boolean shouldRetryOnException(Throwable cause) { - log.error("Not able to locate storage container ", cause); + log.error().exception(cause).log("Not able to locate storage container"); if (cause instanceof StatusRuntimeException || cause instanceof StatusException) { Status status; if (cause instanceof StatusException) { diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/MetaRangeClientImpl.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/MetaRangeClientImpl.java index 7e61b58a236..822087c389f 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/MetaRangeClientImpl.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/MetaRangeClientImpl.java @@ -23,7 +23,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.impl.container.StorageContainerChannel; import org.apache.bookkeeper.clients.impl.container.StorageContainerChannelManager; import org.apache.bookkeeper.clients.impl.internal.api.HashStreamRanges; @@ -37,7 +37,7 @@ /** * A default implementation for {@link MetaRangeClient}. */ -@Slf4j +@CustomLog class MetaRangeClientImpl implements MetaRangeClient { private final StreamProperties streamProps; diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImpl.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImpl.java index a8e26035b03..1e65082971c 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImpl.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImpl.java @@ -31,7 +31,7 @@ import com.google.common.annotations.VisibleForTesting; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.exceptions.ClientException; import org.apache.bookkeeper.clients.impl.container.StorageContainerChannel; import org.apache.bookkeeper.clients.impl.container.StorageContainerChannelManager; @@ -58,7 +58,7 @@ /** * A default implementation for {@link RootRangeClient}. */ -@Slf4j +@CustomLog class RootRangeClientImpl implements RootRangeClient { private final ScheduledExecutorService executor; diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImplWithRetries.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImplWithRetries.java index 2e7dddd784e..0ea57661542 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImplWithRetries.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImplWithRetries.java @@ -25,7 +25,7 @@ import java.util.concurrent.CompletableFuture; import java.util.function.Predicate; import java.util.function.Supplier; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.impl.internal.api.RootRangeClient; import org.apache.bookkeeper.common.util.Backoff; import org.apache.bookkeeper.common.util.OrderedScheduler; @@ -38,7 +38,7 @@ /** * A root range client wrapper with retries. */ -@Slf4j +@CustomLog class RootRangeClientImplWithRetries implements RootRangeClient { @VisibleForTesting @@ -46,7 +46,7 @@ class RootRangeClientImplWithRetries implements RootRangeClient { cause -> shouldRetryOnException(cause); private static boolean shouldRetryOnException(Throwable cause) { - log.error("Reason for the failure ", cause); + log.error().exception(cause).log("Reason for the failure"); if (cause instanceof StatusRuntimeException || cause instanceof StatusException) { Status status; if (cause instanceof StatusException) { diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/StorageServerClientManagerImpl.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/StorageServerClientManagerImpl.java index 3ae3b6bb1f1..cc845ac1bba 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/StorageServerClientManagerImpl.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/StorageServerClientManagerImpl.java @@ -23,7 +23,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentMap; import java.util.function.Function; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.bookkeeper.clients.impl.channel.StorageServerChannel; import org.apache.bookkeeper.clients.impl.channel.StorageServerChannelManager; @@ -44,7 +44,7 @@ /** * A gRPC based {@link StorageServerClientManager} implementation. */ -@Slf4j +@CustomLog public class StorageServerClientManagerImpl extends AbstractAutoAsyncCloseable implements StorageServerClientManager { diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/routing/RangeRouter.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/routing/RangeRouter.java index b1b5bd5c589..f31fb48ae4b 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/routing/RangeRouter.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/routing/RangeRouter.java @@ -25,7 +25,7 @@ import java.util.concurrent.locks.StampedLock; import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.impl.internal.api.HashStreamRanges; import org.apache.bookkeeper.common.router.HashRouter; import org.apache.bookkeeper.stream.proto.RangeProperties; @@ -34,7 +34,7 @@ * A router that used to route the events to ranges. */ @ThreadSafe -@Slf4j +@CustomLog public class RangeRouter { private final HashRouter keyRouter; diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/GrpcChannels.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/GrpcChannels.java index fdd93e31b3d..309f1c3665c 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/GrpcChannels.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/GrpcChannels.java @@ -20,7 +20,7 @@ import io.grpc.ManagedChannelBuilder; import io.grpc.inprocess.InProcessChannelBuilder; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.bookkeeper.common.net.ServiceURI; import org.apache.bookkeeper.common.resolver.NameResolverFactoryProvider; @@ -30,7 +30,7 @@ /** * Utils to create grpc channels. */ -@Slf4j +@CustomLog public final class GrpcChannels { private static final String BACKEND_INPROCESS = "inprocess"; diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/ListenableFutureRpcProcessor.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/ListenableFutureRpcProcessor.java index 28ad7ca9bae..10e5f2d996a 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/ListenableFutureRpcProcessor.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/ListenableFutureRpcProcessor.java @@ -29,7 +29,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.impl.channel.StorageServerChannel; import org.apache.bookkeeper.clients.impl.container.StorageContainerChannel; import org.apache.bookkeeper.common.concurrent.FutureUtils; @@ -38,7 +38,7 @@ /** * A process for processing rpc request on storage container channel. */ -@Slf4j +@CustomLog public abstract class ListenableFutureRpcProcessor implements BiConsumer, FutureCallback, diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestLocationClientImpl.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestLocationClientImpl.java index 665b8ac925e..11f0f25140b 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestLocationClientImpl.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestLocationClientImpl.java @@ -37,7 +37,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.IntStream; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.bookkeeper.clients.exceptions.ClientException; import org.apache.bookkeeper.clients.exceptions.StorageContainerException; @@ -58,7 +58,7 @@ /** * Unit test for {@link LocationClientImpl}. */ -@Slf4j +@CustomLog public class TestLocationClientImpl extends GrpcClientTestBase { private static StorageContainerEndpoint createEndpoint(int groupId) { diff --git a/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufSimpleTableImpl.java b/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufSimpleTableImpl.java index 2ae5ef2e220..596ee93a680 100644 --- a/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufSimpleTableImpl.java +++ b/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufSimpleTableImpl.java @@ -47,7 +47,7 @@ import io.netty.util.ReferenceCountUtil; import java.util.List; import java.util.concurrent.CompletableFuture; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.api.kv.PTable; import org.apache.bookkeeper.api.kv.Txn; import org.apache.bookkeeper.api.kv.impl.op.OpFactoryImpl; @@ -73,7 +73,7 @@ /** * A {@link PTable} implementation using simple grpc calls. */ -@Slf4j +@CustomLog public class PByteBufSimpleTableImpl extends AbstractStub implements PTable { @@ -309,7 +309,7 @@ private void closeResource(AutoCloseable resource) { try { resource.close(); } catch (Exception e) { - log.warn("Fail to close resource {}", resource, e); + log.warn().attr("resource", resource).exception(e).log("Fail to close resource"); } } } diff --git a/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufTableImpl.java b/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufTableImpl.java index 8b84d96e141..3600599a8ce 100644 --- a/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufTableImpl.java +++ b/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufTableImpl.java @@ -26,7 +26,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ScheduledExecutorService; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.api.kv.PTable; import org.apache.bookkeeper.api.kv.Txn; import org.apache.bookkeeper.api.kv.impl.op.OpFactoryImpl; @@ -55,7 +55,7 @@ /** * The default implementation of {@link PTable}. */ -@Slf4j +@CustomLog public class PByteBufTableImpl implements PTable { static final IllegalStateException CAUSE = @@ -228,12 +228,10 @@ CompletableFuture> refreshRangeSpaces(HashStreamRanges // compare the ranges to see if it requires an update HashStreamRanges oldRanges = rangeRouter.getRanges(); if (null != oldRanges && oldRanges.getMaxRangeId() >= newRanges.getMaxRangeId()) { - log.info("No new stream ranges found for stream {}.", streamName); + log.info().attr("streamName", streamName).log("No new stream ranges found for stream."); return FutureUtils.value(this); } - if (log.isInfoEnabled()) { - log.info("Updated the active ranges to {}", newRanges); - } + log.info().attr("newRanges", newRanges).log("Updated the active ranges"); rangeRouter.setRanges(newRanges); // add new ranges Set activeRanges = Sets.newHashSetWithExpectedSize(newRanges.getRanges().size()); @@ -244,9 +242,7 @@ CompletableFuture> refreshRangeSpaces(HashStreamRanges } PTable tableRange = trFactory.openTableRange(props, range, executor, opFactory, resultFactory, kvFactory); - if (log.isInfoEnabled()) { - log.info("Create table range client for range {}", range.getRangeId()); - } + log.info().attr("rangeId", range.getRangeId()).log("Create table range client for range"); this.tableRanges.put(range.getRangeId(), tableRange); }); // remove old ranges diff --git a/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufTableRangeImpl.java b/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufTableRangeImpl.java index a82e2f8fcee..0b25bbd0d65 100644 --- a/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufTableRangeImpl.java +++ b/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufTableRangeImpl.java @@ -24,7 +24,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.api.kv.PTable; import org.apache.bookkeeper.api.kv.Txn; import org.apache.bookkeeper.api.kv.impl.result.KeyValueFactory; @@ -50,7 +50,7 @@ /** * A range of a table. */ -@Slf4j +@CustomLog class PByteBufTableRangeImpl implements PTable { private final long streamId; @@ -263,7 +263,7 @@ private void closeResource(AutoCloseable resource) { try { resource.close(); } catch (Exception e) { - log.warn("Fail to close resource {}", resource, e); + log.warn().attr("resource", resource).exception(e).log("Fail to close resource"); } } } diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/resolver/ServiceNameResolverProvider.java b/stream/common/src/main/java/org/apache/bookkeeper/common/resolver/ServiceNameResolverProvider.java index a3a1b29a6b4..021eee3d597 100644 --- a/stream/common/src/main/java/org/apache/bookkeeper/common/resolver/ServiceNameResolverProvider.java +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/resolver/ServiceNameResolverProvider.java @@ -29,7 +29,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import javax.annotation.Nullable; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.net.ServiceURI; import org.apache.bookkeeper.common.util.SharedResourceManager.Resource; @@ -37,7 +37,7 @@ * An implementation of {@link NameResolverProvider} that provides {@link NameResolver}s * to resolve {@link org.apache.bookkeeper.common.net.ServiceURI}. */ -@Slf4j +@CustomLog public final class ServiceNameResolverProvider extends NameResolverFactoryProvider { private final NameResolverProvider dnsProvider; @@ -76,8 +76,10 @@ public NameResolver newNameResolver(URI targetUri, NameResolver.Args params) { serviceURI = ServiceURI.create(targetUri); } catch (NullPointerException | IllegalArgumentException e) { // invalid uri here, so return null to allow grpc to use other name resolvers - log.info("ServiceNameResolverProvider doesn't know how to resolve {} : cause {}", - targetUri, e.getMessage()); + log.info() + .attr("targetUri", targetUri) + .exceptionMessage(e) + .log("ServiceNameResolverProvider doesn't know how to resolve target URI"); return null; } diff --git a/stream/common/src/main/java/org/apache/bookkeeper/common/util/AbstractAutoAsyncCloseable.java b/stream/common/src/main/java/org/apache/bookkeeper/common/util/AbstractAutoAsyncCloseable.java index 45bff63e1b3..b9853e20275 100644 --- a/stream/common/src/main/java/org/apache/bookkeeper/common/util/AbstractAutoAsyncCloseable.java +++ b/stream/common/src/main/java/org/apache/bookkeeper/common/util/AbstractAutoAsyncCloseable.java @@ -20,13 +20,13 @@ import java.util.concurrent.CompletableFuture; import javax.annotation.concurrent.GuardedBy; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; /** * A common implementation for {@link AutoAsyncCloseable}. */ -@Slf4j +@CustomLog public abstract class AbstractAutoAsyncCloseable implements AutoAsyncCloseable { @GuardedBy("this") diff --git a/stream/common/src/test/java/org/apache/bookkeeper/common/router/HashRouterTest.java b/stream/common/src/test/java/org/apache/bookkeeper/common/router/HashRouterTest.java index 4f46cb05b1a..5c90a25c1dd 100644 --- a/stream/common/src/test/java/org/apache/bookkeeper/common/router/HashRouterTest.java +++ b/stream/common/src/test/java/org/apache/bookkeeper/common/router/HashRouterTest.java @@ -21,14 +21,14 @@ import com.google.common.hash.Hashing; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.hash.Murmur3; import org.junit.Test; /** * Unit test {@link HashRouter}s. */ -@Slf4j +@CustomLog public class HashRouterTest { @Test @@ -52,7 +52,10 @@ public void testByteBufHashRouter() { long[] hash128 = Murmur3.hash128( key, key.readerIndex(), key.readableBytes(), AbstractHashRouter.HASH_SEED); long[] bytesHash128 = Murmur3.hash128(keyBytes, 0, keyBytes.length, AbstractHashRouter.HASH_SEED); - log.info("hash128: {}, bytes hash128: {}", hash128, bytesHash128); + log.info() + .attr("hash128", hash128) + .attr("bytesHash128", bytesHash128) + .log("hash128"); long guavaHash128 = Hashing.murmur3_128((int) AbstractHashRouter.HASH_SEED) .newHasher() .putString("foo", UTF_8) @@ -63,7 +66,9 @@ public void testByteBufHashRouter() { ByteBufHashRouter router = ByteBufHashRouter.of(); long routingHash = router.getRoutingKey(key); - log.info("Routing hash = {}", routingHash); + log.info() + .attr("routingHash", routingHash) + .log("Routing hash"); assertEquals(hash128[0], routingHash); BytesHashRouter bytesRouter = BytesHashRouter.of(); long bytesRoutingHash = bytesRouter.getRoutingKey(keyBytes); diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConfigurationSubscription.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConfigurationSubscription.java index e2c4da22649..479a6369824 100644 --- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConfigurationSubscription.java +++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConfigurationSubscription.java @@ -31,6 +31,7 @@ import java.util.concurrent.CopyOnWriteArraySet; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.commons.configuration2.FileBasedConfiguration; import org.apache.commons.configuration2.PropertiesConfiguration; import org.apache.commons.configuration2.builder.ConfigurationBuilderEvent; @@ -41,8 +42,6 @@ import org.apache.commons.configuration2.ex.ConfigurationException; import org.apache.commons.configuration2.reloading.PeriodicReloadingTrigger; import org.apache.commons.configuration2.reloading.ReloadingController; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * ConfigurationSubscription publishes a reloading, thread-safe view of file configuration. The class @@ -55,8 +54,8 @@ * 2. The underlying FileConfiguration is not at all thread-safe, so its important to ensure access * to this object is always single threaded. */ +@CustomLog public class ConfigurationSubscription implements AutoCloseable { - static final Logger LOG = LoggerFactory.getLogger(ConfigurationSubscription.class); private final ConcurrentBaseConfiguration viewConfig; private final ScheduledExecutorService executorService; @@ -117,7 +116,7 @@ private void initConfig(List configFiles) { try { configChanged(configFile, event); } catch (ConfigurationException e) { - LOG.error("Config reload failed for file {}", configFile, e); + log.error().attr("configFile", configFile).exception(e).log("Config reload failed for file"); } }); FileBasedConfiguration fileConfig = configBuilder.getConfiguration(); @@ -125,7 +124,7 @@ private void initConfig(List configFiles) { } } catch (ConfigurationException ex) { if (!fileNotFound(ex)) { - LOG.error("Config init failed", ex); + log.error().exception(ex).log("Config init failed"); } } } @@ -144,10 +143,8 @@ synchronized void refreshConfiguration() { for (Map.Entry entry : fileConfigs.entrySet()) { File configFile = entry.getKey(); FileBasedConfiguration fileConfig = entry.getValue(); - if (LOG.isDebugEnabled()) { - LOG.debug("Check and reload config, file={}, lastModified={}", configFile, - configFile.lastModified()); - } + log.debug().attr("file", configFile).attr("lastModified", configFile.lastModified()) + .log("Check and reload config"); // load keys Iterator keyIter = fileConfig.getKeys(); while (keyIter.hasNext()) { @@ -163,7 +160,7 @@ synchronized void refreshConfiguration() { clearViewProperty(key); } } - LOG.info("Reload features : {}", confKeys); + log.info().attr("features", confKeys).log("Reload features"); // load keys from files for (Map.Entry entry : fileConfigs.entrySet()) { File configFile = entry.getKey(); @@ -172,7 +169,7 @@ synchronized void refreshConfiguration() { loadView(fileConfig); } catch (Exception ex) { if (!fileNotFound(ex)) { - LOG.error("Config reload failed for file {}", configFile, ex); + log.error().attr("configFile", configFile).exception(ex).log("Config reload failed for file"); } } } @@ -195,9 +192,7 @@ private void loadView(FileBasedConfiguration fileConfig) { } private void clearViewProperty(String key) { - if (LOG.isDebugEnabled()) { - LOG.debug("Removing property, key={}", key); - } + log.debug().attr("key", key).log("Removing property"); viewConfig.clearProperty(key); } @@ -205,9 +200,7 @@ private void setViewProperty(FileBasedConfiguration fileConfig, String key, Object value) { if (!viewConfig.containsKey(key) || !viewConfig.getProperty(key).equals(value)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Setting property, key={} value={}", key, fileConfig.getProperty(key)); - } + log.debug().attr("key", key).attr("value", fileConfig.getProperty(key)).log("Setting property"); viewConfig.setProperty(key, fileConfig.getProperty(key)); } } diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/util/SchedulerUtils.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/util/SchedulerUtils.java index e85db394d47..d284aa7a2ac 100644 --- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/util/SchedulerUtils.java +++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/util/SchedulerUtils.java @@ -19,12 +19,12 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; /** * Scheduler related utils. */ -@Slf4j +@CustomLog public class SchedulerUtils { public static void shutdownScheduler(ExecutorService service, long timeout, TimeUnit timeUnit) { @@ -36,7 +36,7 @@ public static void shutdownScheduler(ExecutorService service, long timeout, Time service.awaitTermination(timeout, timeUnit); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - log.warn("Interrupted when shutting down scheduler : ", e); + log.warn().exception(e).log("Interrupted when shutting down scheduler"); } service.shutdownNow(); } diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/Abortables.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/Abortables.java index 9a34de41133..ad747c3d508 100644 --- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/Abortables.java +++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/Abortables.java @@ -23,7 +23,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import javax.annotation.Nullable; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.distributedlog.common.functions.VoidFunctions; @@ -32,7 +32,7 @@ * * @since 0.3.32 */ -@Slf4j +@CustomLog public final class Abortables { private Abortables() {} @@ -82,7 +82,8 @@ public static void abort(@Nullable Abortable abortable, abortable.abort(); } catch (IOException ioe) { if (swallowIOException) { - log.warn("IOException thrown while aborting Abortable {} : ", abortable, ioe); + log.warn().attr("abortable", abortable).exception(ioe) + .log("IOException thrown while aborting Abortable"); } else { throw ioe; } @@ -108,7 +109,8 @@ public static void abort(@Nullable AsyncAbortable abortable, FutureUtils.result(abortable.asyncAbort()); } catch (Exception e) { if (swallowIOException) { - log.warn("IOException thrown while aborting Abortable {} : ", abortable, e); + log.warn().attr("abortable", abortable).exception(e) + .log("IOException thrown while aborting Abortable"); } else { if (e instanceof IOException) { throw (IOException) e; @@ -134,7 +136,8 @@ public static void abortQuietly(@Nullable Abortable abortable) { try { abort(abortable, true); } catch (IOException e) { - log.error("Unexpected IOException thrown while aborting Abortable {} quietly : ", abortable, e); + log.error().attr("abortable", abortable).exception(e) + .log("Unexpected IOException thrown while aborting Abortable quietly"); } } @@ -153,7 +156,8 @@ public static void abortQuietly(@Nullable AsyncAbortable abortable) { try { abort(abortable, true); } catch (IOException e) { - log.error("Unexpected IOException thrown while aborting Abortable {} quietly : ", abortable, e); + log.error().attr("abortable", abortable).exception(e) + .log("Unexpected IOException thrown while aborting Abortable quietly"); } } diff --git a/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/config/PropertiesWriter.java b/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/config/PropertiesWriter.java index 2835641e3c9..2127eb97f35 100644 --- a/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/config/PropertiesWriter.java +++ b/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/config/PropertiesWriter.java @@ -20,14 +20,13 @@ import java.io.File; import java.io.FileOutputStream; import java.util.Properties; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import lombok.CustomLog; /** * Writer to write properties to files. */ +@CustomLog public class PropertiesWriter { - static final Logger LOG = LoggerFactory.getLogger(PropertiesWriter.class); final FileOutputStream outputStream; final File configFile; @@ -60,9 +59,7 @@ public void save() throws Exception { try (FileOutputStream outputStream = new FileOutputStream(configFile)) { properties.store(outputStream, null); configFile.setLastModified(configFile.lastModified() + 1000); - if (LOG.isDebugEnabled()) { - LOG.debug("save modified={}", configFile.lastModified()); - } + log.debug().attr("modified", configFile.lastModified()).log("save"); } } diff --git a/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/config/TestConfigurationSubscription.java b/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/config/TestConfigurationSubscription.java index ffe8ccdb3b3..7a988955f3b 100644 --- a/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/config/TestConfigurationSubscription.java +++ b/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/config/TestConfigurationSubscription.java @@ -28,20 +28,19 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import lombok.CustomLog; import org.apache.commons.configuration2.CompositeConfiguration; import org.apache.commons.configuration2.event.ConfigurationEvent; import org.jmock.lib.concurrent.DeterministicScheduler; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Notes: * 1. lastModified granularity is platform dependent, generally 1 sec, so we can't wait 1ms for things to * get picked up. */ +@CustomLog public class TestConfigurationSubscription { - static final Logger LOG = LoggerFactory.getLogger(TestConfigurationSubscription.class); public static final int RELOAD_PERIOD = 10; /** @@ -135,7 +134,7 @@ public void testExceptionInConfigLoad() throws Exception { final AtomicInteger count = new AtomicInteger(1); conf.addEventListener(ConfigurationEvent.ANY, event -> { - LOG.info("config changed {}", event); + log.info().attr("event", event).log("config changed"); // Throw after so we actually see the update anyway. if (!event.isBeforeUpdate()) { count.getAndIncrement(); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java index 26630682396..9ef6bf2e7b7 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java @@ -21,16 +21,15 @@ import java.io.IOException; import java.io.InputStream; +import lombok.CustomLog; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.api.LogReader; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * AppendOnlyStreamReader. */ +@CustomLog public class AppendOnlyStreamReader extends InputStream { - static final Logger LOG = LoggerFactory.getLogger(AppendOnlyStreamReader.class); private LogRecordWithInputStream currentLogRecord = null; private final DistributedLogManager dlm; @@ -46,10 +45,11 @@ private static class LogRecordWithInputStream { LogRecordWithInputStream(LogRecordWithDLSN logRecord) { checkNotNull(logRecord); - if (LOG.isDebugEnabled()) { - LOG.debug("Got record dlsn = {}, txid = {}, len = {}", - logRecord.getDlsn(), logRecord.getTransactionId(), logRecord.getPayload().length); - } + log.debug() + .attr("dlsn", logRecord.getDlsn()) + .attr("transactionId", logRecord.getTransactionId()) + .attr("length", logRecord.getPayload().length) + .log("Got record"); this.logRecord = logRecord; this.payloadStream = logRecord.getPayLoadInputStream(); @@ -102,9 +102,7 @@ record = reader.readNext(false); if (null != record) { return new LogRecordWithInputStream(record); } else { - if (LOG.isDebugEnabled()) { - LOG.debug("No record"); - } + log.debug("No record"); return null; } } @@ -138,10 +136,10 @@ public int read(byte[] b, int off, int len) throws IOException { return read; } } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Offset saved = {}, persisted = {}", - currentPosition, currentLogRecord.getLogRecord().getTransactionId()); - } + log.debug() + .attr("currentPosition", currentPosition) + .attr("transactionId", currentLogRecord.getLogRecord().getTransactionId()) + .log("Offset saved"); currentPosition += thisread; read += thisread; } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java index eadf83704df..2a5d9d0000a 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java @@ -20,17 +20,16 @@ import java.io.Closeable; import java.io.IOException; import java.util.concurrent.CompletableFuture; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.distributedlog.exceptions.UnexpectedException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * AppendOnlyStreamWriter. */ +@CustomLog public class AppendOnlyStreamWriter implements Closeable { - static final Logger LOG = LoggerFactory.getLogger(AppendOnlyStreamWriter.class); // Use a 1-length array to satisfy Java's inner class reference rules. Use primitive // type because synchronized block is needed anyway. @@ -39,9 +38,7 @@ public class AppendOnlyStreamWriter implements Closeable { long requestPos = 0; public AppendOnlyStreamWriter(BKAsyncLogWriter logWriter, long pos) { - if (LOG.isDebugEnabled()) { - LOG.debug("initialize at position {}", pos); - } + log.debug().attr("pos", pos).log("initialize at position"); this.logWriter = logWriter; this.syncPos[0] = pos; this.requestPos = pos; @@ -60,7 +57,7 @@ public void force(boolean metadata) throws IOException { } catch (IOException ioe) { throw ioe; } catch (Exception ex) { - LOG.error("unexpected exception in AppendOnlyStreamWriter.force ", ex); + log.error().exception(ex).log("unexpected exception in AppendOnlyStreamWriter.force"); throw new UnexpectedException("unexpected exception in AppendOnlyStreamWriter.force", ex); } synchronized (syncPos) { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java index b6101ff7316..b8803128894 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java @@ -25,6 +25,7 @@ import java.util.concurrent.CompletionStage; import java.util.concurrent.TimeUnit; import java.util.function.Function; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.distributedlog.bk.LedgerMetadata; @@ -39,12 +40,9 @@ import org.apache.distributedlog.io.AsyncAbortable; import org.apache.distributedlog.io.AsyncCloseable; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - +@CustomLog abstract class BKAbstractLogWriter implements Closeable, AsyncCloseable, Abortable, AsyncAbortable { - static final Logger LOG = LoggerFactory.getLogger(BKAbstractLogWriter.class); protected final DistributedLogConfiguration conf; private final DynamicDistributedLogConfiguration dynConf; @@ -72,10 +70,10 @@ abstract class BKAbstractLogWriter implements Closeable, AsyncCloseable, Abortab this.conf = conf; this.dynConf = dynConf; this.bkDistributedLogManager = bkdlm; - if (LOG.isDebugEnabled()) { - LOG.debug("Initial retention period for {} : {}", bkdlm.getStreamName(), - TimeUnit.MILLISECONDS.convert(dynConf.getRetentionPeriodHours(), TimeUnit.HOURS)); - } + log.debug() + .attr("streamName", bkdlm.getStreamName()) + .attr("retentionPeriodMs", TimeUnit.MILLISECONDS.convert(dynConf.getRetentionPeriodHours(), TimeUnit.HOURS)) + .log("Initial retention period"); } @VisibleForTesting @@ -194,7 +192,7 @@ public void onSuccess(LogSegmentMetadata segment) { @Override public void onFailure(Throwable cause) { - LOG.error("Completing Log segments encountered exception", cause); + log.error().exception(cause).log("Completing Log segments encountered exception"); complete(cause); } @@ -406,20 +404,23 @@ private CompletableFuture closeOldLogSegmentAndStartNewOneWi // rescue function cause -> { if (cause instanceof LockingException) { - LOG.warn("We lost lock during completeAndClose log segment for {}." - + "Disable ledger rolling until it is recovered : ", - writeHandler.getFullyQualifiedName(), cause); + log.warn() + .attr("fullyQualifiedName", writeHandler.getFullyQualifiedName()) + .exception(cause) + .log("We lost lock during completeAndClose log segment for." + + "Disable ledger rolling until it is recovered"); bkDistributedLogManager.getLogSegmentRollingPermitManager() .disallowObtainPermits(switchPermit); return FutureUtils.value(oldSegmentWriter); } else if (cause instanceof ZKException) { ZKException zke = (ZKException) cause; if (ZKException.isRetryableZKException(zke)) { - LOG.warn("Encountered zookeeper connection issues during completeAndClose " - + "log segment for {}. " - + "Disable ledger rolling until it is recovered : {}", - writeHandler.getFullyQualifiedName(), - zke.getKeeperExceptionCode()); + log.warn() + .attr("fullyQualifiedName", writeHandler.getFullyQualifiedName()) + .attr("keeperExceptionCode", zke.getKeeperExceptionCode()) + .log("Encountered zookeeper connection issues during" + + " completeAndClose log segment for." + + " Disable ledger rolling until it is recovered"); bkDistributedLogManager.getLogSegmentRollingPermitManager() .disallowObtainPermits(switchPermit); return FutureUtils.value(oldSegmentWriter); @@ -447,10 +448,10 @@ private CompletableFuture closeOldLogSegmentAndStartNewOne( // we switch only when we could allocate a new log segment. BKLogSegmentWriter newSegmentWriter = getAllocatedLogWriter(); if (null == newSegmentWriter) { - if (LOG.isDebugEnabled()) { - LOG.debug("Allocating a new log segment from {} for {}.", startTxId, - writeHandler.getFullyQualifiedName()); - } + log.debug() + .attr("startTxId", startTxId) + .attr("fullyQualifiedName", writeHandler.getFullyQualifiedName()) + .log("Allocating a new log segment"); return writeHandler.asyncStartLogSegment(startTxId, bestEffort, allowMaxTxID) .thenCompose(new Function>() { @Override @@ -465,10 +466,10 @@ public CompletableFuture apply(BKLogSegmentWriter newSegment } } cacheAllocatedLogWriter(newSegmentWriter); - if (LOG.isDebugEnabled()) { - LOG.debug("Allocated a new log segment from {} for {}.", startTxId, - writeHandler.getFullyQualifiedName()); - } + log.debug() + .attr("startTxId", startTxId) + .attr("fullyQualifiedName", writeHandler.getFullyQualifiedName()) + .log("Allocated a new log segment"); return completeOldSegmentAndCacheNewLogSegmentWriter(oldSegmentWriter, newSegmentWriter); } }); @@ -534,7 +535,7 @@ public BKLogSegmentWriter apply(BKLogSegmentWriter newSegmentWriter) { protected synchronized void checkClosedOrInError(String operation) throws AlreadyClosedException { if (null != closePromise) { - LOG.error("Executing " + operation + " on already closed Log Writer"); + log.error("Executing " + operation + " on already closed Log Writer"); throw new AlreadyClosedException("Executing " + operation + " on already closed Log Writer"); } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java index 2e9327be420..c69864c48ce 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java @@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.function.Function; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; @@ -46,8 +47,6 @@ import org.apache.distributedlog.exceptions.ReadCancelledException; import org.apache.distributedlog.exceptions.UnexpectedException; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * BookKeeper based {@link AsyncLogReader} implementation. @@ -67,8 +66,8 @@ *
  • `async_reader`/idle_reader_error: counter. the number idle reader errors. * */ +@CustomLog class BKAsyncLogReader implements AsyncLogReader, Runnable, AsyncNotification { - static final Logger LOG = LoggerFactory.getLogger(BKAsyncLogReader.class); private static final Function, LogRecordWithDLSN> READ_NEXT_MAP_FUNCTION = records -> records.get(0); @@ -192,10 +191,10 @@ void addRecord(LogRecordWithDLSN record) { } void complete() { - if (LOG.isTraceEnabled()) { - LOG.trace("{} : Satisfied promise with {} records", - readHandler.getFullyQualifiedName(), records.size()); - } + log.trace() + .attr("fullyQualifiedName", readHandler.getFullyQualifiedName()) + .attr("numRecords", records.size()) + .log(": Satisfied promise with records"); delayUntilPromiseSatisfied.registerSuccessfulEvent( enqueueTime.stop().elapsed(TimeUnit.MICROSECONDS), TimeUnit.MICROSECONDS); Stopwatch stopwatch = Stopwatch.createStarted(); @@ -216,9 +215,7 @@ void complete() { this.scheduler = scheduler; this.readHandler = bkDistributedLogManager.createReadHandler(subscriberId, this, true); - if (LOG.isDebugEnabled()) { - LOG.debug("Starting async reader at {}", startDLSN); - } + log.debug().attr("startDLSN", startDLSN).log("Starting async reader"); this.startDLSN = startDLSN; this.scheduleDelayStopwatch = Stopwatch.createUnstarted(); this.readNextDelayStopwatch = Stopwatch.createStarted(); @@ -307,8 +304,9 @@ void cancelIdleReaderTask() { idleReaderTimeoutTask.cancel(true); } } catch (Exception exc) { - LOG.info("{}: Failed to cancel the background idle reader timeout task", - readHandler.getFullyQualifiedName()); + log.info() + .attr("fullyQualifiedName", readHandler.getFullyQualifiedName()) + .log(": Failed to cancel the background idle reader timeout task"); } } @@ -364,7 +362,7 @@ private boolean checkClosedOrInError(String operation) { Throwable cause = lastExceptionUpdater.get(this); if (null != cause) { - LOG.trace("Cancelling pending reads"); + log.trace("Cancelling pending reads"); cancelAllPendingReads(cause); return true; } @@ -579,13 +577,14 @@ public void run() { Stopwatch runTime = Stopwatch.createStarted(); int iterations = 0; long scheduleCountLocal = scheduleCountUpdater.get(this); - if (LOG.isDebugEnabled()) { - LOG.debug("{}: Scheduled Background Reader", readHandler.getFullyQualifiedName()); - } + log.debug() + .attr("fullyQualifiedName", readHandler.getFullyQualifiedName()) + .log(": Scheduled Background Reader"); while (true) { - if (LOG.isTraceEnabled()) { - LOG.trace("{}: Executing Iteration: {}", readHandler.getFullyQualifiedName(), iterations++); - } + log.trace() + .attr("fullyQualifiedName", readHandler.getFullyQualifiedName()) + .attr("iteration", iterations++) + .log(": Executing Iteration"); PendingReadRequest nextRequest = null; synchronized (this) { @@ -593,7 +592,9 @@ public void run() { // Queue is empty, nothing to read, return if (null == nextRequest) { - LOG.trace("{}: Queue Empty waiting for Input", readHandler.getFullyQualifiedName()); + log.trace() + .attr("fullyQualifiedName", readHandler.getFullyQualifiedName()) + .log(": Queue Empty waiting for Input"); scheduleCountUpdater.set(this, 0); backgroundReaderRunTime.registerSuccessfulEvent( runTime.stop().elapsed(TimeUnit.MICROSECONDS), TimeUnit.MICROSECONDS); @@ -601,8 +602,9 @@ public void run() { } if (disableProcessingReadRequests) { - LOG.info("Reader of {} is forced to stop processing read requests", - readHandler.getFullyQualifiedName()); + log.info() + .attr("fullyQualifiedName", readHandler.getFullyQualifiedName()) + .log("Reader of is forced to stop processing read requests"); return; } } @@ -621,7 +623,10 @@ public void run() { if (checkClosedOrInError("readNext")) { Throwable lastException = lastExceptionUpdater.get(this); if (lastException != null && !(lastException.getCause() instanceof LogNotFoundException)) { - LOG.warn("{}: Exception", readHandler.getFullyQualifiedName(), lastException); + log.warn() + .attr("fullyQualifiedName", readHandler.getFullyQualifiedName()) + .exception(lastException) + .log(": Exception"); } backgroundReaderRunTime.registerFailedEvent( runTime.stop().elapsed(TimeUnit.MICROSECONDS), TimeUnit.MICROSECONDS); @@ -665,8 +670,10 @@ record = readNextRecord(); } catch (IOException exc) { setLastException(exc); if (!(exc instanceof LogNotFoundException)) { - LOG.warn("{} : read with skip Exception", - readHandler.getFullyQualifiedName(), lastExceptionUpdater.get(this)); + log.warn() + .attr("fullyQualifiedName", readHandler.getFullyQualifiedName()) + .exception(lastExceptionUpdater.get(this)) + .log(": read with skip Exception"); } continue; } @@ -709,7 +716,7 @@ record = readNextRecord(); } } else { if (0 == scheduleCountLocal) { - LOG.trace("Schedule count dropping to zero", lastExceptionUpdater.get(this)); + log.trace().exception(lastExceptionUpdater.get(this)).log("Schedule count dropping to zero"); backgroundReaderRunTime.registerSuccessfulEvent( runTime.stop().elapsed(TimeUnit.MICROSECONDS), TimeUnit.MICROSECONDS); return; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java index da7723a383b..438351dfe33 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java @@ -27,6 +27,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.feature.Feature; @@ -42,8 +43,6 @@ import org.apache.distributedlog.feature.CoreFeatureKeys; import org.apache.distributedlog.util.FailpointUtils; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * BookKeeper based {@link AsyncLogWriter} implementation. @@ -64,10 +63,9 @@ * * See {@link BKLogSegmentWriter} for segment writer stats. */ +@CustomLog class BKAsyncLogWriter extends BKAbstractLogWriter implements AsyncLogWriter { - static final Logger LOG = LoggerFactory.getLogger(BKAsyncLogWriter.class); - static Function, Boolean> truncationResultConverter = segments -> true; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java index 3505cbe1a55..60cf11a889a 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java @@ -32,6 +32,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.function.Function; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; @@ -69,10 +70,6 @@ import org.apache.distributedlog.util.Allocator; import org.apache.distributedlog.util.DLUtils; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** *

    Metrics

    @@ -91,8 +88,8 @@ * for details. * */ +@CustomLog class BKDistributedLogManager implements DistributedLogManager { - static final Logger LOG = LoggerFactory.getLogger(BKDistributedLogManager.class); static final Function RECORD_2_TXID_FUNCTION = record -> record.getTransactionId(); @@ -330,8 +327,6 @@ synchronized BKLogReadHandler createReadHandler(Optional subscriberId, // Create Ledger Allocator - - // Create Write Handler public BKLogWriteHandler createWriteHandler(boolean lockHandler) @@ -765,14 +760,19 @@ public CompletableFuture apply(Void complete) { if (fromDLSN.isPresent()) { return FutureUtils.value(reader); } - LOG.info("Reader {} @ {} reading last commit position from subscription store after acquired lock.", - subscriberId.get(), name); + log.info() + .attr("subscriberId", subscriberId.get()) + .attr("name", name) + .log("Reader @ reading last commit position from subscription store after acquired lock."); // we acquired lock final SubscriptionsStore subscriptionsStore = driver.getSubscriptionsStore(getStreamName()); return subscriptionsStore.getLastCommitPosition(subscriberId.get()) .thenCompose(lastCommitPosition -> { - LOG.info("Reader {} @ {} positioned to last commit position {}.", - subscriberId.get(), name, lastCommitPosition); + log.info() + .attr("subscriberId", subscriberId.get()) + .attr("name", name) + .attr("lastCommitPosition", lastCommitPosition) + .log("Reader @ positioned to last commit position."); try { reader.setStartDLSN(lastCommitPosition); } catch (UnexpectedException e) { @@ -820,7 +820,7 @@ LogReader getInputStreamInternal(long fromTxnId) LogReader getInputStreamInternal(DLSN fromDLSN, Optional fromTxnId) throws IOException { - LOG.info("Create sync reader starting from {}", fromDLSN); + log.info().attr("fromDLSN", fromDLSN).log("Create sync reader"); checkClosedOrInError("getInputStream"); return new BKSyncLogReader( conf, @@ -1016,7 +1016,10 @@ public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException { checkClosedOrInError("purgeLogSegmentsOlderThan"); BKLogWriteHandler ledgerHandler = createWriteHandler(true); try { - LOG.info("Purging logs for {} older than {}", ledgerHandler.getFullyQualifiedName(), minTxIdToKeep); + log.info() + .attr("fullyQualifiedName", ledgerHandler.getFullyQualifiedName()) + .attr("minTxIdToKeep", minTxIdToKeep) + .log("Purging logs older than threshold"); Utils.ioResult(ledgerHandler.purgeLogSegmentsOlderThanTxnId(minTxIdToKeep)); } finally { Utils.closeQuietly(ledgerHandler); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java index 8ed2cfeb362..7305752c9ff 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java @@ -28,6 +28,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.feature.FeatureProvider; @@ -47,8 +48,6 @@ import org.apache.distributedlog.namespace.NamespaceDriver; import org.apache.distributedlog.util.ConfUtils; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * BKDistributedLogNamespace is the default implementation of {@link Namespace}. It uses @@ -80,8 +79,8 @@ *

    DistributedLogManager

    * All the core stats about reader and writer are exposed under current scope via {@link BKDistributedLogManager}. */ +@CustomLog public class BKDistributedLogNamespace implements Namespace { - static final Logger LOG = LoggerFactory.getLogger(BKDistributedLogNamespace.class); private final String clientId; private final int regionId; @@ -321,7 +320,7 @@ protected DistributedLogManager openLogInternal( */ private void checkState() throws IOException { if (closed.get()) { - LOG.error("BK namespace {} is already closed", namespace); + log.error().attr("namespace", namespace).log("BK namespace is already closed"); throw new AlreadyClosedException("BK namespace " + namespace + " is already closed"); } } @@ -342,6 +341,6 @@ public void close() { // Shutdown the schedulers SchedulerUtils.shutdownScheduler(scheduler, conf.getSchedulerShutdownTimeoutMs(), TimeUnit.MILLISECONDS); - LOG.info("Executor Service Stopped."); + log.info("Executor Service Stopped."); } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogHandler.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogHandler.java index 4d666509a42..2304d98d67c 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogHandler.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogHandler.java @@ -33,6 +33,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.function.Function; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; @@ -55,8 +56,6 @@ import org.apache.distributedlog.logsegment.PerStreamLogSegmentCache; import org.apache.distributedlog.metadata.LogMetadata; import org.apache.distributedlog.metadata.LogStreamMetadataStore; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * The base class about log handler on managing log segments. @@ -80,8 +79,8 @@ * @see BKLogWriteHandler * @see BKLogReadHandler */ +@CustomLog abstract class BKLogHandler implements AsyncCloseable, AsyncAbortable { - static final Logger LOG = LoggerFactory.getLogger(BKLogHandler.class); protected final LogMetadata logMetadata; protected final DistributedLogConfiguration conf; @@ -481,8 +480,10 @@ protected List getCachedLogSegments(Comparator 0) { if (elapsedMillis > metadataLatencyWarnThresholdMillis) { - LOG.warn("{} received inprogress log segment in {} millis: {}", - getFullyQualifiedName(), elapsedMillis, metadata); + log.warn() + .attr("stream", getFullyQualifiedName()) + .attr("elapsedMillis", elapsedMillis) + .attr("metadata", metadata) + .log("received inprogress log segment in millis"); } getInprogressSegmentStat.registerSuccessfulEvent(elapsedMicroSec, TimeUnit.MICROSECONDS); } else { @@ -526,8 +530,11 @@ protected void addLogSegmentToCache(String name, LogSegmentMetadata metadata) { long elapsedMicroSec = TimeUnit.MILLISECONDS.toMicros(elapsedMillis); if (elapsedMicroSec > 0) { if (elapsedMillis > metadataLatencyWarnThresholdMillis) { - LOG.warn("{} received completed log segment in {} millis : {}", - getFullyQualifiedName(), elapsedMillis, metadata); + log.warn() + .attr("stream", getFullyQualifiedName()) + .attr("elapsedMillis", elapsedMillis) + .attr("metadata", metadata) + .log("received completed log segment in millis"); } getCompletedSegmentStat.registerSuccessfulEvent(elapsedMicroSec, TimeUnit.MICROSECONDS); } else { @@ -618,9 +625,7 @@ protected void readLogSegmentsFromStore(final Versioned> logSegment removedSegments.addAll(segmentChanges.getRight()); if (segmentsAdded.isEmpty()) { - if (LOG.isTraceEnabled()) { - LOG.trace("No segments added for {}.", getFullyQualifiedName()); - } + log.trace().attr("stream", getFullyQualifiedName()).log("No segments added"); // update the cache before #getCachedLogSegments to return updateLogSegmentCache(removedSegments, addedSegments); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java index 167915a7f02..3a3962f59d6 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java @@ -28,6 +28,7 @@ import java.util.concurrent.CopyOnWriteArraySet; import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; @@ -51,8 +52,6 @@ import org.apache.distributedlog.metadata.LogMetadataForReader; import org.apache.distributedlog.metadata.LogStreamMetadataStore; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Log Handler for Readers. @@ -93,8 +92,8 @@ * All read lock related stats are exposed under scope `read_lock`. * for detail stats. */ +@CustomLog class BKLogReadHandler extends BKLogHandler implements LogSegmentNamesListener { - static final Logger LOG = LoggerFactory.getLogger(BKLogReadHandler.class); protected final LogMetadataForReader logMetadataForReader; @@ -169,7 +168,10 @@ synchronized CompletableFuture lockStream() { .thenCompose(lock -> { try { BKLogReadHandler.this.readLock = lock; - LOG.info("acquiring readlock {} at {}", getLockClientId(), getReadLockPath()); + log.info() + .attr("lockClientId", getLockClientId()) + .attr("readLockPath", getReadLockPath()) + .log("Acquiring readlock"); return acquireLockOnExecutorThread(lock); } catch (LockingException le) { return FutureUtils.exception(le); @@ -199,13 +201,20 @@ CompletableFuture acquireLockOnExecutorThread(DistributedLock lock) throws acquireFuture.whenCompleteAsync(new FutureEventListener() { @Override public void onSuccess(DistributedLock lock) { - LOG.info("acquired readlock {} at {}", getLockClientId(), getReadLockPath()); + log.info() + .attr("lockClientId", getLockClientId()) + .attr("readLockPath", getReadLockPath()) + .log("Acquired readlock"); threadAcquirePromise.complete(null); } @Override public void onFailure(Throwable cause) { - LOG.info("failed to acquire readlock {} at {}", getLockClientId(), getReadLockPath(), cause); + log.info() + .attr("lockClientId", getLockClientId()) + .attr("readLockPath", getReadLockPath()) + .exception(cause) + .log("Failed to acquire readlock"); threadAcquirePromise.completeExceptionally(cause); } }); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java index 7ac75b99279..dc0f53bea3f 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java @@ -39,6 +39,7 @@ import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Function; +import lombok.CustomLog; import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.AsyncCallback.CloseCallback; import org.apache.bookkeeper.client.BKException; @@ -80,8 +81,6 @@ import org.apache.distributedlog.util.FailpointUtils; import org.apache.distributedlog.util.SimplePermitLimiter; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * BookKeeper Based Log Segment Writer. @@ -104,8 +103,8 @@ *
  • transmit/outstanding/requests: per stream gauge. the number of outstanding transmits each stream. * */ +@CustomLog class BKLogSegmentWriter implements LogSegmentWriter, AddCallback, Runnable, Sizable { - static final Logger LOG = LoggerFactory.getLogger(BKLogSegmentWriter.class); final Writer REJECT_WRITES_WRITER = new Writer() { @Override @@ -332,8 +331,11 @@ public Number getSample() { final int configuredTransmissionThreshold = dynConf.getOutputBufferSize(); if (configuredTransmissionThreshold > MAX_LOGRECORDSET_SIZE) { - LOG.warn("Setting output buffer size {} greater than max transmission size {} for log segment {}", - configuredTransmissionThreshold, MAX_LOGRECORDSET_SIZE, fullyQualifiedLogSegment); + log.warn() + .attr("configuredTransmissionThreshold", configuredTransmissionThreshold) + .attr("MAX_LOGRECORDSET_SIZE", MAX_LOGRECORDSET_SIZE) + .attr("fullyQualifiedLogSegment", fullyQualifiedLogSegment) + .log("Setting output buffer size greater than max transmission size for log segment"); this.transmissionThreshold = MAX_LOGRECORDSET_SIZE; } else { this.transmissionThreshold = configuredTransmissionThreshold; @@ -534,7 +536,10 @@ private synchronized void abortPacket(BKTransmitPacket packet) { Throwable reason = new WriteCancelledException(streamName, Utils.transmitException(rc)); recordSet.abortTransmit(reason); } - LOG.info("Stream {} aborted {} writes", fullyQualifiedLogSegment, numRecords); + log.info() + .attr("fullyQualifiedLogSegment", fullyQualifiedLogSegment) + .attr("numRecords", numRecords) + .log("Stream aborted writes"); } private synchronized long getWritesPendingTransmit() { @@ -569,7 +574,9 @@ private void closeInternal(final boolean abort, // Cancel the periodic keep alive schedule first if (null != periodicKeepAliveSchedule) { if (!periodicKeepAliveSchedule.cancel(false)) { - LOG.info("Periodic keepalive for log segment {} isn't cancelled.", getFullyQualifiedLogSegment()); + log.info() + .attr("logSegment", getFullyQualifiedLogSegment()) + .log("Periodic keepalive for log segment isn't cancelled."); } } @@ -580,14 +587,16 @@ private void closeInternal(final boolean abort, // be cancelled, it means that it is doing flushing. So following flushes would be synchronized // to wait until background flush completed. if (!periodicFlushSchedule.cancel(false)) { - LOG.info("Periodic flush for log segment {} isn't cancelled.", getFullyQualifiedLogSegment()); + log.info() + .attr("logSegment", getFullyQualifiedLogSegment()) + .log("Periodic flush for log segment isn't cancelled."); } } // If it is a normal close and the stream isn't in an error state, we attempt to flush any buffered data if (!abort && !isLogSegmentInError()) { this.enforceLock = false; - LOG.info("Flushing before closing log segment {}", getFullyQualifiedLogSegment()); + log.info().attr("logSegment", getFullyQualifiedLogSegment()).log("Flushing before closing log segment"); flushAndCommit().whenComplete(new FutureEventListener() { @Override public void onSuccess(Long value) { @@ -609,10 +618,13 @@ public void onFailure(Throwable cause) { private void abortTransmitPacketOnClose(final boolean abort, final MutableObject throwExc, final CompletableFuture closePromise) { - LOG.info("Closing BKPerStreamLogWriter (abort={}) for {} :" - + " lastDLSN = {} outstandingTransmits = {} writesPendingTransmit = {}", - abort, fullyQualifiedLogSegment, getLastDLSN(), - outstandingTransmitsUpdater.get(this), getWritesPendingTransmit()); + log.info() + .attr("abort", abort) + .attr("fullyQualifiedLogSegment", fullyQualifiedLogSegment) + .attr("lastDLSN", getLastDLSN()) + .attr("outstandingTransmits", outstandingTransmitsUpdater.get(this)) + .attr("writesPendingTransmit", getWritesPendingTransmit()) + .log("Closing BKPerStreamLogWriter"); // Save the current packet to reset, leave a new empty packet to avoid a race with // addCompleteDeferredProcessing. @@ -634,7 +646,7 @@ public void onSuccess(Integer transmitResult) { } @Override public void onFailure(Throwable cause) { - LOG.error("Unexpected error on transmit completion ", cause); + log.error().exception(cause).log("Unexpected error on transmit completion"); } }); } else { @@ -722,7 +734,7 @@ public synchronized CompletableFuture asyncWrite(LogRecord record, boolean // We may incorrectly report transmit failure here, but only if we happened to hit // packet/xmit size limit conditions AND fail flush above, which should happen rarely if (null != result) { - LOG.error("Overriding first result with flush failure {}", result); + log.error().attr("result", result).log("Overriding first result with flush failure"); } result = FutureUtils.exception(ioe); @@ -832,8 +844,11 @@ public synchronized CompletableFuture writeInternal(LogRecord record) recordSetWriter.writeRecord(record, writePromise); if (record.getTransactionId() < lastTxId) { - LOG.info("Log Segment {} TxId decreased Last: {} Record: {}", - fullyQualifiedLogSegment, lastTxId, record.getTransactionId()); + log.info() + .attr("fullyQualifiedLogSegment", fullyQualifiedLogSegment) + .attr("lastTxId", lastTxId) + .attr("transactionId", record.getTransactionId()) + .log("Log Segment TxId decreased Last: Record"); } if (!record.isControl()) { // only update last tx id for user records @@ -989,8 +1004,10 @@ void flushIfNeededNoThrow() { try { flushIfNeeded(); } catch (IOException ioe) { - LOG.error("Encountered exception while flushing log records to stream {}", - fullyQualifiedLogSegment, ioe); + log.error() + .attr("fullyQualifiedLogSegment", fullyQualifiedLogSegment) + .exception(ioe) + .log("Encountered exception while flushing log records to stream"); } } @@ -1012,7 +1029,7 @@ public void run() { scheduledFlushExceptionUpdater.set(BKLogSegmentWriter.this, null); } catch (Exception exc) { scheduledFlushExceptionUpdater.set(BKLogSegmentWriter.this, exc); - LOG.error("Delayed flush failed", exc); + log.error().exception(exc).log("Delayed flush failed"); } } } @@ -1090,9 +1107,10 @@ private CompletableFuture transmit() // stream has encountered an error and cannot be written to. if (!transmitResultUpdater.compareAndSet(this, BKException.Code.OK, BKException.Code.OK)) { - LOG.error("Log Segment {} Trying to write to an errored stream; Error is {}", - fullyQualifiedLogSegment, - BKException.getMessage(transmitResultUpdater.get(this))); + log.error() + .attr("fullyQualifiedLogSegment", fullyQualifiedLogSegment) + .attr("error", BKException.getMessage(transmitResultUpdater.get(this))) + .log("Log Segment trying to write to an errored stream"); throw new BKTransmitException("Trying to write to an errored stream;" + " Error code : (" + transmitResultUpdater.get(this) + ") " + BKException.getMessage(transmitResultUpdater.get(this)), transmitResultUpdater.get(this)); @@ -1122,8 +1140,10 @@ private CompletableFuture transmit() if (e instanceof InvalidEnvelopedEntryException) { alertStatsLogger.raise("Invalid enveloped entry for segment {} : ", fullyQualifiedLogSegment, e); } - LOG.error("Exception while enveloping entries for segment: {}", - new Object[] {fullyQualifiedLogSegment}, e); + log.error() + .attr("logSegment", fullyQualifiedLogSegment) + .exception(e) + .log("Exception while enveloping entries for segment"); // If a write fails here, we need to set the transmit result to an error so that // no future writes go through and violate ordering guarantees. transmitResultUpdater.set(this, BKException.Code.WriteException); @@ -1187,14 +1207,19 @@ public void addComplete(final int rc, LedgerHandle handle, // Sanity check to make sure we're receiving these callbacks in order. if (entryId > -1 && lastEntryId >= entryId) { - LOG.error("Log segment {} saw out of order entry {} lastEntryId {}", - fullyQualifiedLogSegment, entryId, lastEntryId); + log.error() + .attr("fullyQualifiedLogSegment", fullyQualifiedLogSegment) + .attr("entryId", entryId) + .attr("lastEntryId", lastEntryId) + .log("Log segment saw out of order entry lastEntryId"); } lastEntryId = entryId; if (!(ctx instanceof BKTransmitPacket)) { - LOG.error("Log segment {} received addComplete with invalid context {}", - fullyQualifiedLogSegment, ctx); + log.error() + .attr("fullyQualifiedLogSegment", fullyQualifiedLogSegment) + .attr("ctx", ctx) + .log("Log segment received addComplete with invalid context"); return; } final BKTransmitPacket transmitPacket = (BKTransmitPacket) ctx; @@ -1240,9 +1265,13 @@ public void onSuccess(Void done) { } @Override public void onFailure(Throwable cause) { - LOG.error("addComplete processing failed for {} entry {} lastTxId {} rc {} with error", - fullyQualifiedLogSegment, entryId, - transmitPacket.getRecordSet().getMaxTxId(), rc, cause); + log.error() + .attr("fullyQualifiedLogSegment", fullyQualifiedLogSegment) + .attr("entryId", entryId) + .attr("maxTxId", transmitPacket.getRecordSet().getMaxTxId()) + .attr("rc", rc) + .exception(cause) + .log("addComplete processing failed"); } }, directExecutor()); // Race condition if we notify before the addComplete is enqueued. @@ -1269,8 +1298,12 @@ private void addCompleteDeferredProcessing(final BKTransmitPacket transmitPacket // be enqueued; they will be failed with WriteException cancelPendingPromises = (BKException.Code.OK != rc); } else { - LOG.warn("Log segment {} entryId {}: Tried to set transmit result to ({}) but is already ({})", - fullyQualifiedLogSegment, entryId, rc, transmitResultUpdater.get(this)); + log.warn() + .attr("fullyQualifiedLogSegment", fullyQualifiedLogSegment) + .attr("entryId", entryId) + .attr("rc", rc) + .attr("transmitResult", transmitResultUpdater.get(this)) + .log("Tried to set transmit result but it is already set"); } if (transmitResultUpdater.get(this) != BKException.Code.OK) { @@ -1343,10 +1376,9 @@ public synchronized void run() { private synchronized void backgroundFlush(boolean controlFlushOnly) { if (null != closeFuture) { // if the log segment is closing, skip any background flushing - if (LOG.isDebugEnabled()) { - LOG.debug("Skip background flushing since log segment {} is closing.", - getFullyQualifiedLogSegment()); - } + log.debug() + .attr("logSegment", getFullyQualifiedLogSegment()) + .log("Skip background flushing since log segment is closing."); return; } try { @@ -1366,17 +1398,19 @@ private synchronized void backgroundFlush(boolean controlFlushOnly) { pFlushMisses.inc(); } } catch (IOException exc) { - LOG.error("Log Segment {}: Error encountered by the periodic flush", fullyQualifiedLogSegment, exc); + log.error() + .attr("fullyQualifiedLogSegment", fullyQualifiedLogSegment) + .exception(exc) + .log("Log Segment: Error encountered by the periodic flush"); } } private synchronized void keepAlive() { if (null != closeFuture) { // if the log segment is closing, skip sending any keep alive records. - if (LOG.isDebugEnabled()) { - LOG.debug("Skip sending keepAlive control record since log segment {} is closing.", - getFullyQualifiedLogSegment()); - } + log.debug() + .attr("logSegment", getFullyQualifiedLogSegment()) + .log("Skip sending keepAlive control record since log segment is closing."); return; } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java index 7fa5e398207..644c3858d84 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java @@ -31,6 +31,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.function.Function; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; @@ -66,8 +67,6 @@ import org.apache.distributedlog.util.FailpointUtils; import org.apache.distributedlog.util.Transaction; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Log Handler for Writers. @@ -81,8 +80,8 @@ *
  • `segments`/delete : opstats. latency characteristics on deleting a log segment. * */ +@CustomLog class BKLogWriteHandler extends BKLogHandler { - static final Logger LOG = LoggerFactory.getLogger(BKLogWriteHandler.class); private static Transaction.OpListener NULL_OP_LISTENER = new Transaction.OpListener() { @@ -124,7 +123,10 @@ public void onAbort(Throwable t) { new Function, CompletableFuture>() { @Override public CompletableFuture apply(List segmentList) { - LOG.info("Initiating Recovery For {} : {}", getFullyQualifiedName(), segmentList); + log.info() + .attr("stream", getFullyQualifiedName()) + .attr("segmentList", segmentList) + .log("Initiating Recovery For"); // if lastLedgerRollingTimeMillis is not updated, we set it to now. synchronized (BKLogWriteHandler.this) { if (lastLedgerRollingTimeMillis < 0) { @@ -493,16 +495,20 @@ protected long assignLogSegmentSequenceNumber() throws IOException { && (DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO == maxLogSegmentSequenceNo.getSequenceNumber())) { // no ledger seqno stored in /ledgers before - LOG.info("No max ledger sequence number found while creating log segment {} for {}.", - logSegmentSeqNo, getFullyQualifiedName()); + log.info() + .attr("logSegmentSeqNo", logSegmentSeqNo) + .attr("stream", getFullyQualifiedName()) + .log("No max ledger sequence number found while creating log segment"); } else if (maxLogSegmentSequenceNo.getSequenceNumber() + 1 != logSegmentSeqNo // case 1 && maxLogSegmentSequenceNo.getSequenceNumber() != logSegmentSeqNo) { // case 2 // case 1 is the common case, where the new log segment is 1 more than the previous // case 2 can occur when the writer crashes with an empty in progress ledger. This is then deleted // on recovery, so the next new segment will have a matching sequence number - LOG.warn("Unexpected max log segment sequence number {} for {} : list of cached segments = {}", - maxLogSegmentSequenceNo.getSequenceNumber(), getFullyQualifiedName(), - getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR)); + log.warn() + .attr("sequenceNumber", maxLogSegmentSequenceNo.getSequenceNumber()) + .attr("stream", getFullyQualifiedName()) + .attr("cachedLogSegments", getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR)) + .log("Unexpected max log segment sequence number"); // there is max log segment number recorded there and it isn't match. throw exception. throw new DLIllegalStateException("Unexpected max log segment sequence number " + maxLogSegmentSequenceNo.getSequenceNumber() + " for " + getFullyQualifiedName() @@ -555,12 +561,15 @@ protected void doStartLogSegment(final long txId, long highestTxIdWritten = maxTxId.get(); if (txId < highestTxIdWritten) { if (highestTxIdWritten == DistributedLogConstants.MAX_TXID) { - LOG.error("We've already marked the stream as ended and attempting to start a new log segment"); + log.error("We've already marked the stream as ended and attempting to start a new log segment"); FutureUtils.completeExceptionally(promise, new EndOfStreamException("Writing to a stream after it has been marked as completed")); return; } else { - LOG.error("We've already seen TxId {} the max TXId is {}", txId, highestTxIdWritten); + log.error() + .attr("txId", txId) + .attr("highestTxIdWritten", highestTxIdWritten) + .log("Already seen this TxId"); FutureUtils.completeExceptionally(promise, new TransactionIdOutOfOrderException(txId, highestTxIdWritten)); return; @@ -652,10 +661,10 @@ private void createInprogressLogSegment(Transaction txn, writeLogSegment(txn, l); // Try storing max sequence number. - if (LOG.isDebugEnabled()) { - LOG.debug("Try storing max sequence number in startLogSegment {} : {}", - inprogressZnodePath, logSegmentSeqNo); - } + log.debug() + .attr("inprogressZnodePath", inprogressZnodePath) + .attr("logSegmentSeqNo", logSegmentSeqNo) + .log("Try storing max sequence number in startLogSegment"); storeMaxSequenceNumber(txn, maxLogSegmentSequenceNo, logSegmentSeqNo, true); txn.execute().whenCompleteAsync(new FutureEventListener() { @@ -892,9 +901,7 @@ private void doCompleteAndCloseLogSegmentAfterLogSegmentListFetched( return; } - if (LOG.isDebugEnabled()) { - LOG.debug("Completing and Closing Log Segment {} {}", firstTxId, lastTxId); - } + log.debug().attr("firstTxId", firstTxId).attr("lastTxId", lastTxId).log("Completing and Closing Log Segment"); LogSegmentMetadata inprogressLogSegment = readLogSegmentFromCache(inprogressZnodeName); // validate log segment @@ -943,11 +950,16 @@ private void doCompleteAndCloseLogSegmentAfterLogSegmentListFetched( || (maxLogSegmentSequenceNo.getSequenceNumber() == logSegmentSeqNo + 1)) { // ignore the case that a new inprogress log segment is pre-allocated // before completing current inprogress one - LOG.info("Try storing max sequence number {} in completing {}.", - new Object[] { logSegmentSeqNo, inprogressLogSegment.getZkPath() }); + log.info() + .attr("logSegmentSeqNo", logSegmentSeqNo) + .attr("zkPath", inprogressLogSegment.getZkPath()) + .log("Try storing max sequence number in completing."); } else { - LOG.warn("Unexpected max ledger sequence number {} found while completing log segment {} for {}", - maxLogSegmentSequenceNo.getSequenceNumber(), logSegmentSeqNo, getFullyQualifiedName()); + log.warn() + .attr("sequenceNumber", maxLogSegmentSequenceNo.getSequenceNumber()) + .attr("logSegmentSeqNo", logSegmentSeqNo) + .attr("stream", getFullyQualifiedName()) + .log("Unexpected max ledger sequence number found while completing log segment"); if (validateLogSegmentSequenceNumber) { FutureUtils.completeExceptionally(promise, new DLIllegalStateException("Unexpected max log segment sequence number " @@ -987,18 +999,21 @@ private void doCompleteAndCloseLogSegmentAfterLogSegmentListFetched( // store max sequence number storeMaxSequenceNumber(txn, maxLogSegmentSequenceNo, maxSeqNo, false); // update max txn id. - if (LOG.isDebugEnabled()) { - LOG.debug("Trying storing LastTxId in Finalize Path {} LastTxId {}", - pathForCompletedLedger, lastTxId); - } + log.debug() + .attr("pathForCompletedLedger", pathForCompletedLedger) + .attr("lastTxId", lastTxId) + .log("Trying storing LastTxId in Finalize Path LastTxId"); storeMaxTxId(txn, maxTxId, lastTxId); txn.execute().whenCompleteAsync(new FutureEventListener() { @Override public void onSuccess(Void value) { - LOG.info("Completed {} to {} for {} : {}", - inprogressZnodeName, completedLogSegment.getSegmentName(), - getFullyQualifiedName(), completedLogSegment); + log.info() + .attr("inprogressZnodeName", inprogressZnodeName) + .attr("segmentName", completedLogSegment.getSegmentName()) + .attr("stream", getFullyQualifiedName()) + .attr("completedLogSegment", completedLogSegment) + .log("Completed"); FutureUtils.complete(promise, completedLogSegment); } @@ -1027,14 +1042,20 @@ public CompletableFuture apply(final LogSegmentMetadata l) { return FutureUtils.value(l); } - LOG.info("Recovering last record in log segment {} for {}.", l, getFullyQualifiedName()); + log.info() + .attr("logSegment", l) + .attr("stream", getFullyQualifiedName()) + .log("Recovering last record in log segment"); return asyncReadLastRecord(l, true, true, true).thenCompose( lastRecord -> completeLogSegment(l, lastRecord)); } private CompletableFuture completeLogSegment(LogSegmentMetadata l, LogRecordWithDLSN lastRecord) { - LOG.info("Recovered last record in log segment {} for {}.", l, getFullyQualifiedName()); + log.info() + .attr("logSegment", l) + .attr("stream", getFullyQualifiedName()) + .log("Recovered last record in log segment"); long endTxId = DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID; int recordCount = 0; @@ -1049,13 +1070,13 @@ private CompletableFuture completeLogSegment(LogSegmentMetad } if (endTxId == DistributedLogConstants.INVALID_TXID) { - LOG.error("Unrecoverable corruption has occurred in segment " + log.error("Unrecoverable corruption has occurred in segment " + l.toString() + " at path " + l.getZkPath() + ". Unable to continue recovery."); return FutureUtils.exception(new IOException("Unrecoverable corruption," + " please check logs.")); } else if (endTxId == DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID) { - LOG.info("Inprogress segment {} is empty, deleting", l); + log.info().attr("logSegment", l).log("Inprogress segment is empty, deleting"); return deleteLogSegment(l).thenApply( (result) -> { @@ -1093,31 +1114,37 @@ CompletableFuture> setLogSegmentsOlderThanDLSNTruncated private CompletableFuture> setLogSegmentsOlderThanDLSNTruncated( List logSegments, final DLSN dlsn) { - if (LOG.isDebugEnabled()) { - LOG.debug("Setting truncation status on logs older than {} from {} for {}", - dlsn, logSegments, getFullyQualifiedName()); - } + log.debug() + .attr("dlsn", dlsn) + .attr("logSegments", logSegments) + .attr("stream", getFullyQualifiedName()) + .log("Setting truncation status on logs older than threshold"); List truncateList = new ArrayList(logSegments.size()); LogSegmentMetadata partialTruncate = null; - LOG.info("{}: Truncating log segments older than {}", getFullyQualifiedName(), dlsn); + log.info() + .attr("stream", getFullyQualifiedName()) + .attr("dlsn", dlsn) + .log("Truncating log segments older than threshold"); for (int i = 0; i < logSegments.size(); i++) { LogSegmentMetadata l = logSegments.get(i); if (!l.isInProgress()) { if (l.getLastDLSN().compareTo(dlsn) < 0) { - if (LOG.isDebugEnabled()) { - LOG.debug("{}: Truncating log segment {} ", getFullyQualifiedName(), l); - } + log.debug().attr("stream", getFullyQualifiedName()).attr("logSegment", l) + .log("Truncating log segment"); truncateList.add(l); } else if (l.getFirstDLSN().compareTo(dlsn) < 0) { // Can be satisfied by at most one segment if (null != partialTruncate) { String logMsg = String.format("Potential metadata inconsistency for stream %s at segment %s", getFullyQualifiedName(), l); - LOG.error(logMsg); + log.error(logMsg); return FutureUtils.exception(new DLIllegalStateException(logMsg)); } - LOG.info("{}: Partially truncating log segment {} older than {}.", - getFullyQualifiedName(), l, dlsn); + log.info() + .attr("stream", getFullyQualifiedName()) + .attr("logSegment", l) + .attr("dlsn", dlsn) + .log("Partially truncating log segment"); partialTruncate = l; } else { break; @@ -1173,8 +1200,11 @@ public CompletableFuture> apply(List> setLogSegmentTruncationStatu private CompletableFuture> deleteLogSegments( final List logs) { - if (LOG.isTraceEnabled()) { - LOG.trace("Purging logs for {} : {}", getFullyQualifiedName(), logs); - } + log.trace().attr("stream", getFullyQualifiedName()).attr("logs", logs).log("Purging logs"); return FutureUtils.processList(logs, segment -> deleteLogSegment(segment), scheduler); } private CompletableFuture deleteLogSegment( final LogSegmentMetadata ledgerMetadata) { - LOG.info("Deleting ledger {} for {}", ledgerMetadata, getFullyQualifiedName()); + log.info() + .attr("ledgerMetadata", ledgerMetadata) + .attr("stream", getFullyQualifiedName()) + .log("Deleting ledger"); final CompletableFuture promise = new CompletableFuture(); final Stopwatch stopwatch = Stopwatch.createStarted(); promise.whenComplete(new FutureEventListener() { @@ -1305,8 +1336,11 @@ public void onAbort(Throwable t) { promise.complete(segmentMetadata); return; } else { - LOG.error("Couldn't purge {} for {}: with error {}", - segmentMetadata, getFullyQualifiedName(), t); + log.error() + .attr("segmentMetadata", segmentMetadata) + .attr("stream", getFullyQualifiedName()) + .exception(t) + .log("Couldn't purge log segment"); promise.completeExceptionally(t); } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java index dcd38d42469..85b72da628d 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java @@ -19,10 +19,12 @@ import java.io.IOException; import java.util.List; +import lombok.CustomLog; import org.apache.distributedlog.api.LogWriter; import org.apache.distributedlog.config.DynamicDistributedLogConfiguration; import org.apache.distributedlog.util.Utils; +@CustomLog class BKSyncLogWriter extends BKAbstractLogWriter implements LogWriter { public BKSyncLogWriter(DistributedLogConfiguration conf, @@ -89,18 +91,14 @@ public long flush() throws IOException { public long commit() throws IOException { checkClosedOrInError("commit"); - if (LOG.isDebugEnabled()) { - LOG.debug("FlushAndSync Started"); - } + log.debug("FlushAndSync Started"); long highestTransactionId = 0; BKLogSegmentWriter writer = getCachedLogWriter(); if (null != writer) { highestTransactionId = Math.max(highestTransactionId, Utils.ioResult(writer.commit())); - if (LOG.isDebugEnabled()) { - LOG.debug("FlushAndSync Completed"); - } - } else if (LOG.isDebugEnabled()) { - LOG.debug("FlushAndSync Completed - Nothing to Flush"); + log.debug("FlushAndSync Completed"); + } else { + log.debug("FlushAndSync Completed - Nothing to Flush"); } return highestTransactionId; } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java index 8f5004aed08..2e5ef1254ef 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java @@ -25,6 +25,7 @@ import java.util.Collections; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import lombok.CustomLog; import org.apache.bookkeeper.client.AsyncCallback; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BKException.Code; @@ -46,10 +47,6 @@ import org.apache.distributedlog.exceptions.DLInterruptedException; import org.apache.distributedlog.net.NetUtils; import org.apache.distributedlog.util.ConfUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * BookKeeper Client wrapper over {@link BookKeeper}. @@ -59,8 +56,8 @@ *
  • bookkeeper operation stats are exposed under current scope by {@link BookKeeper} * */ +@CustomLog public class BookKeeperClient { - static final Logger LOG = LoggerFactory.getLogger(BookKeeperClient.class); // Parameters to build bookkeeper client private final DistributedLogConfiguration conf; @@ -104,7 +101,7 @@ private synchronized void commonInitialization( try { dnsResolverCls = conf.getEnsemblePlacementDnsResolverClass(); } catch (ConfigurationException e) { - LOG.error("Failed to load bk dns resolver : ", e); + log.error().exception(e).log("Failed to load bk dns resolver"); throw new IOException("Failed to load bk dns resolver : ", e); } final DNSToSwitchMapping dnsResolver = @@ -176,23 +173,28 @@ private synchronized void initialize() throws IOException { } if (ownZK) { - LOG.info("BookKeeper Client created {} with its own ZK Client : ledgersPath = {}, numRetries = {}, " - + "sessionTimeout = {}, backoff = {}, maxBackoff = {}, dnsResolver = {}", - name, ledgersPath, - conf.getBKClientZKNumRetries(), conf.getBKClientZKSessionTimeoutMilliSeconds(), - conf.getBKClientZKRetryBackoffStartMillis(), conf.getBKClientZKRetryBackoffMaxMillis(), - conf.getBkDNSResolverOverrides()); + log.info() + .attr("name", name) + .attr("ledgersPath", ledgersPath) + .attr("bKClientZKNumRetries", conf.getBKClientZKNumRetries()) + .attr("bKClientZKSessionTimeoutMilliSeconds", conf.getBKClientZKSessionTimeoutMilliSeconds()) + .attr("bKClientZKRetryBackoffStartMillis", conf.getBKClientZKRetryBackoffStartMillis()) + .attr("bKClientZKRetryBackoffMaxMillis", conf.getBKClientZKRetryBackoffMaxMillis()) + .attr("bkDNSResolverOverrides", conf.getBkDNSResolverOverrides()) + .log("BookKeeper Client created with its own ZK Client"); } else { - LOG.info("BookKeeper Client created {} with shared zookeeper client : ledgersPath = {}, numRetries = {}, " - + "sessionTimeout = {}, backoff = {}, maxBackoff = {}, dnsResolver = {}", - name, ledgersPath, - conf.getZKNumRetries(), conf.getZKSessionTimeoutMilliseconds(), - conf.getZKRetryBackoffStartMillis(), conf.getZKRetryBackoffMaxMillis(), - conf.getBkDNSResolverOverrides()); + log.info() + .attr("name", name) + .attr("ledgersPath", ledgersPath) + .attr("zKNumRetries", conf.getZKNumRetries()) + .attr("zKSessionTimeoutMilliseconds", conf.getZKSessionTimeoutMilliseconds()) + .attr("zKRetryBackoffStartMillis", conf.getZKRetryBackoffStartMillis()) + .attr("zKRetryBackoffMaxMillis", conf.getZKRetryBackoffMaxMillis()) + .attr("bkDNSResolverOverrides", conf.getBkDNSResolverOverrides()) + .log("BookKeeper Client created with shared zookeeper client"); } } - public synchronized BookKeeper get() throws IOException { checkClosedOrInError(); if (null == bkc) { @@ -267,16 +269,16 @@ public void close() { zkcToClose = zkc; } - LOG.info("BookKeeper Client closed {}", name); + log.info().attr("name", name).log("BookKeeper Client closed"); if (null != bkcToClose) { try { bkcToClose.close(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOG.warn("Interrupted on closing bookkeeper client {} : ", name, e); + log.warn().attr("name", name).exception(e).log("Interrupted on closing bookkeeper client"); Thread.currentThread().interrupt(); } catch (BKException e) { - LOG.warn("Error on closing bookkeeper client {} : ", name, e); + log.warn().attr("name", name).exception(e).log("Error on closing bookkeeper client"); } } if (null != zkcToClose) { @@ -288,7 +290,7 @@ public void close() { public synchronized void checkClosedOrInError() throws AlreadyClosedException { if (closed) { - LOG.error("BookKeeper Client {} is already closed", name); + log.error().attr("name", name).log("BookKeeper Client is already closed"); throw new AlreadyClosedException("BookKeeper Client " + name + " is already closed"); } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java index 02db226e555..335dc2db26f 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java @@ -25,6 +25,7 @@ import java.util.Iterator; import java.util.Optional; import java.util.Set; +import lombok.CustomLog; import org.apache.bookkeeper.common.conf.ConfigurationUtil; import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.bookkeeper.conf.ClientConfiguration; @@ -42,8 +43,6 @@ import org.apache.distributedlog.feature.DefaultFeatureProvider; import org.apache.distributedlog.net.DNSResolverForRacks; import org.apache.distributedlog.net.DNSResolverForRows; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * DistributedLog Configuration. @@ -78,8 +77,8 @@ * * @see org.apache.bookkeeper.conf.ClientConfiguration */ +@CustomLog public class DistributedLogConfiguration extends CompositeConfiguration { - static final Logger LOG = LoggerFactory.getLogger(DistributedLogConfiguration.class); private static ClassLoader defaultLoader; @@ -556,9 +555,10 @@ public void loadStreamConf(Optional streamConfigura ignoredSettings.add(setting); } } - if (LOG.isWarnEnabled() && !ignoredSettings.isEmpty()) { - LOG.warn("invalid stream configuration override(s): {}", - StringUtils.join(ignoredSettings, ";")); + if (!ignoredSettings.isEmpty()) { + log.warn() + .attr("ignoredSettings", StringUtils.join(ignoredSettings, ";")) + .log("invalid stream configuration override(s)"); } } @@ -3590,5 +3590,4 @@ public void validate() { } } - } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java index 491027b550a..3166f897679 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java @@ -34,6 +34,7 @@ import java.util.LinkedList; import java.util.List; import java.util.concurrent.CompletableFuture; +import lombok.CustomLog; import org.apache.distributedlog.Entry.Writer; import org.apache.distributedlog.exceptions.InvalidEnvelopedEntryException; import org.apache.distributedlog.exceptions.LogRecordTooLongException; @@ -41,16 +42,13 @@ import org.apache.distributedlog.io.CompressionCodec; import org.apache.distributedlog.io.CompressionCodec.Type; import org.apache.distributedlog.io.CompressionUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * {@link ByteBuf} based log record set writer. */ +@CustomLog class EnvelopedEntryWriter implements Writer { - private static final Logger logger = LoggerFactory.getLogger(EnvelopedEntryWriter.class); - private static class WriteRequest { private final int numRecords; @@ -117,8 +115,7 @@ public synchronized void writeRecord(LogRecord record, writeRequests.add(new WriteRequest(numRecords, transmitPromise)); maxTxId = Math.max(maxTxId, record.getTransactionId()); } catch (IOException e) { - logger.error("Failed to append record to record set of {} : ", - logName, e); + log.error().attr("logName", logName).exception(e).log("Failed to append record to record set"); throw new WriteException(logName, "Failed to append record to record set of " + logName); } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java index b9a6bb5c4ab..e5a006f4bff 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java @@ -26,6 +26,7 @@ import java.util.Optional; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim; import org.apache.bookkeeper.util.IOUtils; @@ -38,16 +39,13 @@ import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooKeeper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Utility class for setting up bookkeeper ensembles * and bringing individual bookies up and down. */ +@CustomLog public class LocalDLMEmulator { - private static final Logger LOG = LoggerFactory.getLogger(LocalDLMEmulator.class); public static final String DLOG_NAMESPACE = "/messaging/distributedlog"; @@ -141,9 +139,12 @@ private LocalDLMEmulator(final int numBookies, final boolean shouldStartZK, public void run() { try { try { - LOG.info("Starting {} bookies : allowLoopback = {}", numBookies, serverConf.getAllowLoopback()); + log.info() + .attr("numBookies", numBookies) + .attr("allowLoopback", serverConf.getAllowLoopback()) + .log("Starting bookies"); lb.start(); - LOG.info("{} bookies are started.", numBookies); + log.info().attr("numBookies", numBookies).log("bookies are started."); while (true) { Thread.sleep(1000); } @@ -154,7 +155,7 @@ public void run() { Thread.currentThread().interrupt(); // go away quietly } catch (Exception e) { - LOG.error("Error starting local bk", e); + log.error().exception(e).log("Error starting local bk"); } } }; @@ -167,8 +168,10 @@ public void start() throws Exception { } int bookiesUp = checkBookiesUp(numBookies, zkTimeoutSec); if (numBookies != bookiesUp) { - LOG.info("Only {} bookies are up, expected {} bookies to be there.", - bookiesUp, numBookies); + log.info() + .attr("bookiesUp", bookiesUp) + .attr("numBookies", numBookies) + .log("Only bookies are up, expected bookies to be there."); } assert (numBookies == bookiesUp); // Provision "/messaging/distributedlog" namespace @@ -210,13 +213,13 @@ public int checkBookiesUp(int count, int timeout) throws Exception { false); children.remove("readonly"); mostRecentSize = children.size(); - if ((mostRecentSize > count) || LOG.isDebugEnabled()) { - LOG.info("Found " + mostRecentSize + " bookies up, " - + "waiting for " + count); - if ((mostRecentSize > count) || LOG.isTraceEnabled()) { - for (String child : children) { - LOG.info(" server: " + child); - } + if (mostRecentSize > count) { + log.info() + .attr("bookiesUp", mostRecentSize) + .attr("waitingFor", count) + .log("Found bookies up"); + for (String child : children) { + log.info().attr("server", child).log("server"); } } if (mostRecentSize == count) { @@ -299,7 +302,7 @@ public static Pair runZookeeperOnAnyPort(int baseP while (!success) { try { - LOG.info("zk trying to bind to port " + zkPort); + log.info("zk trying to bind to port " + zkPort); zks = LocalBookKeeper.runZookeeper(1000, zkPort, zkDir); success = true; } catch (BindException be) { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java index 7ea416201ed..0418c96d6ab 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.util.Comparator; import java.util.concurrent.CompletableFuture; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.distributedlog.exceptions.DLInterruptedException; import org.apache.distributedlog.exceptions.LogSegmentNotFoundException; @@ -35,16 +36,13 @@ import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Utility class for storing the metadata associated * with a single edit log segment, stored in a single ledger. */ +@CustomLog public class LogSegmentMetadata { - static final Logger LOG = LoggerFactory.getLogger(LogSegmentMetadata.class); /** * LogSegmentMetadataVersion. */ @@ -365,7 +363,6 @@ public int compare(LogSegmentMetadata o1, } } - } }; @@ -642,7 +639,7 @@ public void processResult(int rc, String path, Object ctx, byte[] data, Stat sta LogSegmentMetadata metadata = parseData(path, data, skipMinVersionCheck); FutureUtils.complete(result, metadata); } catch (IOException ie) { - LOG.error("Error on parsing log segment metadata from {} : ", path, ie); + log.error().attr("path", path).exception(ie).log("Error on parsing log segment metadata"); result.completeExceptionally(ie); } } @@ -1018,7 +1015,7 @@ public void write(ZooKeeperClient zkc) Thread.currentThread().interrupt(); throw new DLInterruptedException("Interrupted on creating ledger znode " + zkPath, ie); } catch (Exception e) { - LOG.error("Error creating ledger znode {}", zkPath, e); + log.error().attr("zkPath", zkPath).exception(e).log("Error creating ledger znode"); throw new IOException("Error creating ledger znode " + zkPath); } } @@ -1026,9 +1023,7 @@ public void write(ZooKeeperClient zkc) boolean checkEquivalence(ZooKeeperClient zkc, String path) { try { LogSegmentMetadata other = FutureUtils.result(read(zkc, path)); - if (LOG.isTraceEnabled()) { - LOG.trace("Verifying {} against {}", this, other); - } + log.trace().attr("this", this).attr("other", other).log("Verifying against"); boolean retVal; @@ -1048,12 +1043,16 @@ boolean checkEquivalence(ZooKeeperClient zkc, String path) { } if (!retVal) { - LOG.warn("Equivalence check failed between {} and {}", this, other); + log.warn().attr("this", this).attr("other", other).log("Equivalence check failed between and"); } return retVal; } catch (Exception e) { - LOG.error("Could not check equivalence between:" + this + " and data in " + path, e); + log.error() + .attr("this", this) + .attr("path", path) + .exception(e) + .log("Could not check equivalence"); return false; } } @@ -1108,7 +1107,6 @@ public Mutator mutator() { return new Mutator(this); } - // // Version Checking Utilities // diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/MaxTxId.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/MaxTxId.java index f7a9102dd15..019258c96bf 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/MaxTxId.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/MaxTxId.java @@ -17,18 +17,17 @@ */ package org.apache.distributedlog; +import lombok.CustomLog; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; import org.apache.distributedlog.util.DLUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Utility class for storing and reading * the max seen txid in zookeeper. */ +@CustomLog class MaxTxId { - static final Logger LOG = LoggerFactory.getLogger(MaxTxId.class); private Version version; private long currentMax; @@ -41,7 +40,7 @@ class MaxTxId { try { this.currentMax = DLUtils.deserializeTransactionId(maxTxIdData.getValue()); } catch (NumberFormatException e) { - LOG.warn("Invalid txn id stored in ", e); + log.warn().exception(e).log("Invalid txn id stored"); this.currentMax = DistributedLogConstants.INVALID_TXID; } } else { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java index 5ed3f3c4d90..00efcc9d8e6 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java @@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.Collectors; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; @@ -51,8 +52,6 @@ import org.apache.distributedlog.logsegment.LogSegmentEntryReader; import org.apache.distributedlog.logsegment.LogSegmentEntryStore; import org.apache.distributedlog.logsegment.LogSegmentFilter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * New ReadAhead Reader that uses {@link org.apache.distributedlog.logsegment.LogSegmentEntryReader}. @@ -60,14 +59,13 @@ *

    NOTE: all the state changes happen in the same thread. All *unsafe* methods should be submitted to the order * scheduler using stream name as the key.

    */ +@CustomLog class ReadAheadEntryReader implements AsyncCloseable, LogSegmentListener, LogSegmentEntryReader.StateChangeListener, FutureEventListener> { - private static final Logger logger = LoggerFactory.getLogger(ReadAheadEntryReader.class); - private enum State { IDLE, READING, @@ -225,7 +223,7 @@ public void run() { try { safeRun(); } catch (Throwable cause) { - logger.error("Caught unexpected exception : ", cause); + log.error().exception(cause).log("Caught unexpected exception"); } } @@ -406,18 +404,22 @@ private void orderedSubmit(Runnable runnable) { try { scheduler.executeOrdered(streamName, runnable); } catch (RejectedExecutionException ree) { - logger.debug("Failed to submit and execute an operation for readhead entry reader of {}", - streamName, ree); + log.debug() + .attr("streamName", streamName) + .exception(ree) + .log("Failed to submit and execute an operation for readhead entry reader"); } } public void start(final List segmentList) { // Managed to get 5mil character long log line from here. // Will limit the output. - logger.info("Starting the readahead entry reader for {} : number of segments: {}, top 10 segments = {}", - readHandler.getFullyQualifiedName(), segmentList.size(), - segmentList.size() > 10 - ? segmentList.stream().limit(10).collect(Collectors.toList()) : segmentList); + log.info() + .attr("fullyQualifiedName", readHandler.getFullyQualifiedName()) + .attr("numberOfSegments", segmentList.size()) + .attr("topSegments", segmentList.size() > 10 + ? segmentList.stream().limit(10).collect(Collectors.toList()) : segmentList) + .log("Starting the readahead entry reader"); started.set(true); processLogSegments(segmentList); } @@ -465,8 +467,10 @@ public CompletableFuture asyncClose() { try { scheduler.executeOrdered(streamName, () -> unsafeAsyncClose(closeFuture)); } catch (RejectedExecutionException ree) { - logger.warn("Scheduler has been shutdown before closing the readahead entry reader for stream {}", - streamName, ree); + log.warn() + .attr("streamName", streamName) + .exception(ree) + .log("Scheduler has been shutdown before closing the readahead entry reader for stream"); unsafeAsyncClose(closeFuture); } @@ -474,9 +478,7 @@ public CompletableFuture asyncClose() { } private void unsafeAsyncClose(CompletableFuture closePromise) { - if (logger.isDebugEnabled()) { - logger.debug("[{}][state:{}] Closing read ahead", streamName, state); - } + log.debug().attr("streamName", streamName).attr("state", state).log("Closing read ahead"); state = State.CLOSED; List> closeFutures = Lists.newArrayListWithExpectedSize( @@ -525,7 +527,7 @@ private void notifyStateChangeOnFailure(Throwable cause) { void setLastException(IOException cause) { if (!lastException.compareAndSet(null, cause)) { - logger.debug("last exception has already been set to ", lastException.get()); + log.debug().exception(lastException.get()).log("Last exception has already been set"); } // the exception is set and notify the state change notifyStateChangeOnFailure(cause); @@ -533,9 +535,10 @@ void setLastException(IOException cause) { orderedSubmit(new CloseableRunnable() { @Override public void safeRun() { - if (logger.isDebugEnabled()) { - logger.debug("[{}][state:{}] Read ahead errored", streamName, state); - } + log.debug() + .attr("streamName", streamName) + .attr("state", state) + .log("Read ahead errored"); state = State.ERROR; } }); @@ -551,9 +554,11 @@ void checkCatchingUpStatus(LogSegmentEntryReader reader) { if (reader.getSegment().isInProgress() && isCatchingUp && reader.hasCaughtUpOnInprogress()) { - logger.info("ReadAhead for {} is caught up at entry {} @ log segment {}.", - readHandler.getFullyQualifiedName(), - reader.getLastAddConfirmed(), reader.getSegment()); + log.info() + .attr("fullyQualifiedName", readHandler.getFullyQualifiedName()) + .attr("lastAddConfirmed", reader.getLastAddConfirmed()) + .attr("segment", reader.getSegment()) + .log("ReadAhead for is caught up at entry @ log segment."); isCatchingUp = false; } } @@ -561,7 +566,9 @@ void checkCatchingUpStatus(LogSegmentEntryReader reader) { void markCaughtup() { if (isCatchingUp) { isCatchingUp = false; - logger.info("ReadAhead for {} is caught up", readHandler.getFullyQualifiedName()); + log.info() + .attr("fullyQualifiedName", readHandler.getFullyQualifiedName()) + .log("ReadAhead for is caught up"); } } @@ -707,9 +714,11 @@ private void unsafeProcessLogSegments(List segments) { private boolean updateLogSegmentMetadata(SegmentReader reader, LogSegmentMetadata newMetadata) { if (reader.getSegment().getLogSegmentSequenceNumber() != newMetadata.getLogSegmentSequenceNumber()) { - logger.error("Inconsistent state found in entry reader for {} : " - + "current segment = {}, new segment = {}", - streamName, reader.getSegment(), newMetadata); + log.error() + .attr("streamName", streamName) + .attr("segment", reader.getSegment()) + .attr("newMetadata", newMetadata) + .log("Inconsistent state found in entry reader"); setLastException(new DLIllegalStateException("Inconsistent state found in entry reader for " + streamName + " : current segment = " + reader.getSegment() + ", new segment = " + newMetadata)); return false; @@ -730,7 +739,7 @@ private boolean updateLogSegmentMetadata(SegmentReader reader, * Reinitialize the log segments. */ private void unsafeReinitializeLogSegments(List segments) { - logger.info("Reinitialize log segments with {}", segments); + log.info().attr("segments", segments).log("Reinitialize log segments"); int segmentIdx = 0; for (; segmentIdx < segments.size(); segmentIdx++) { LogSegmentMetadata segment = segments.get(segmentIdx); @@ -749,9 +758,11 @@ private void unsafeReinitializeLogSegments(List segments) { } } else { if (currentSegmentSequenceNumber != segment.getLogSegmentSequenceNumber()) { - logger.error("Inconsistent state found in entry reader for {} : " - + "current segment sn = {}, new segment sn = {}", - streamName, currentSegmentSequenceNumber, segment.getLogSegmentSequenceNumber()); + log.error() + .attr("streamName", streamName) + .attr("currentSegmentSequenceNumber", currentSegmentSequenceNumber) + .attr("logSegmentSequenceNumber", segment.getLogSegmentSequenceNumber()) + .log("Inconsistent state found in entry reader"); setLastException(new DLIllegalStateException("Inconsistent state found in entry reader for " + streamName + " : current segment sn = " + currentSegmentSequenceNumber + ", new segment sn = " + segment.getLogSegmentSequenceNumber())); @@ -829,7 +840,10 @@ private void unsafeInitializeLogSegments(List segments) { } skipTruncatedLogSegments = false; if (!isAllowedToPosition(segment, dlsnToStart)) { - logger.error("segment {} is not allowed to position at {}", segment, dlsnToStart); + log.error() + .attr("segment", segment) + .attr("dlsnToStart", dlsnToStart) + .log("Segment is not allowed to position"); return; } @@ -893,8 +907,11 @@ private boolean isAllowedToPosition(LogSegmentMetadata segment, DLSN fromDLSN) { fromDLSN, segment); } if (!conf.getIgnoreTruncationStatus()) { - logger.error("{}: Trying to position reader on {} when {} is marked partially truncated", - streamName, fromDLSN, segment); + log.error() + .attr("streamName", streamName) + .attr("fromDLSN", fromDLSN) + .attr("segment", segment) + .log(": Trying to position reader on when is marked partially truncated"); setLastException(new AlreadyTruncatedTransactionException(streamName + " : trying to position read ahead at " + fromDLSN @@ -924,13 +941,13 @@ private void unsafeMoveToNextLogSegment() { FutureUtils.ensure( currentSegmentReader.close(), removeClosedSegmentReadersFunc); - logger.debug("close current segment reader {}", currentSegmentReader.getSegment()); + log.debug().attr("segment", currentSegmentReader.getSegment()).log("close current segment reader"); currentSegmentReader = null; } boolean hasSegmentToRead = false; if (null != nextSegmentReader) { currentSegmentReader = nextSegmentReader; - logger.debug("move to read segment {}", currentSegmentReader.getSegment()); + log.debug().attr("segment", currentSegmentReader.getSegment()).log("move to read segment"); currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber(); nextSegmentReader = null; // start reading @@ -940,7 +957,7 @@ private void unsafeMoveToNextLogSegment() { unsafePrefetchNextSegment(false); if (null != nextSegmentReader) { currentSegmentReader = nextSegmentReader; - logger.debug("move to read segment {}", currentSegmentReader.getSegment()); + log.debug().attr("segment", currentSegmentReader.getSegment()).log("move to read segment"); currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber(); nextSegmentReader = null; unsafePrefetchNextSegment(true); @@ -949,8 +966,9 @@ private void unsafeMoveToNextLogSegment() { } if (!hasSegmentToRead) { // no more segment to read, wait until new log segment arrive if (isCatchingUp) { - logger.info("ReadAhead for {} is caught up and no log segments to read now", - readHandler.getFullyQualifiedName()); + log.info() + .attr("fullyQualifiedName", readHandler.getFullyQualifiedName()) + .log("ReadAhead for is caught up and no log segments to read now"); isCatchingUp = false; } } @@ -962,9 +980,7 @@ void completeRead() { orderedSubmit(new CloseableRunnable() { @Override public void safeRun() { - if (logger.isDebugEnabled()) { - logger.debug("[{}][state:{}] Read completed", streamName, state); - } + log.debug().attr("streamName", streamName).attr("state", state).log("Read completed"); if (state == State.READING) { state = State.IDLE; } @@ -980,10 +996,12 @@ public void safeRun() { boolean cacheFull = isCacheFull(); SegmentReader reader = currentSegmentReader; boolean hasMoreSegments = reader != null; - if (logger.isDebugEnabled()) { - logger.debug("[{}][state:{}] scheduling read, cacheFull {}, hasMoreSegments {}", - streamName, state, cacheFull, hasMoreSegments); - } + log.debug() + .attr("streamName", streamName) + .attr("state", state) + .attr("cacheFull", cacheFull) + .attr("hasMoreSegments", hasMoreSegments) + .log("Scheduling read"); switch (state) { case IDLE: if (cacheFull || !hasMoreSegments) { @@ -1017,7 +1035,7 @@ public void onSegmentsUpdated(List segments) { if (segments == null || segments.isEmpty()) { return; } - logger.info("segments is updated with {}", segments); + log.info().attr("segments", segments).log("Segments updated"); processLogSegments(segments); } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadUtils.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadUtils.java index c909f83a76b..1c18d3b381e 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadUtils.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadUtils.java @@ -25,6 +25,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.distributedlog.logsegment.LogSegmentEntryStore; @@ -33,16 +34,13 @@ import org.apache.distributedlog.selector.FirstTxIdNotLessThanSelector; import org.apache.distributedlog.selector.LastRecordSelector; import org.apache.distributedlog.selector.LogRecordSelector; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Utility function for readers. */ +@CustomLog public class ReadUtils { - static final Logger LOG = LoggerFactory.getLogger(ReadUtils.class); - private static final int MIN_SEARCH_BATCH_SIZE = 2; // @@ -249,18 +247,22 @@ private static CompletableFuture asyncReadRecordFromEntries( final CompletableFuture promise = new CompletableFuture(); final long startEntryId = context.curStartEntryId.get(); final long endEntryId = context.curEndEntryId.get(); - if (LOG.isDebugEnabled()) { - LOG.debug("{} reading entries [{} - {}] from {}.", - streamName, startEntryId, endEntryId, metadata); - } + log.debug() + .attr("streamName", streamName) + .attr("startEntryId", startEntryId) + .attr("endEntryId", endEntryId) + .attr("metadata", metadata) + .log("Reading entries"); FutureEventListener> readEntriesListener = new FutureEventListener>() { @Override public void onSuccess(final List entries) { - if (LOG.isDebugEnabled()) { - LOG.debug("{} finished reading entries [{} - {}] from {}", - streamName, startEntryId, endEntryId, metadata); - } + log.debug() + .attr("streamName", streamName) + .attr("startEntryId", startEntryId) + .attr("endEntryId", endEntryId) + .attr("metadata", metadata) + .log("Finished reading entries"); for (Entry.Reader entry : entries) { try { visitEntryRecords(entry, context, selector); @@ -273,10 +275,13 @@ public void onSuccess(final List entries) { } LogRecordWithDLSN record = selector.result(); - if (LOG.isDebugEnabled()) { - LOG.debug("{} got record from entries [{} - {}] of {} : {}", - streamName, startEntryId, endEntryId, metadata, record); - } + log.debug() + .attr("streamName", streamName) + .attr("startEntryId", startEntryId) + .attr("endEntryId", endEntryId) + .attr("metadata", metadata) + .attr("record", record) + .log("Got record from entries"); promise.complete(record); } @@ -345,11 +350,13 @@ private static void asyncReadRecordFromEntries( new FutureEventListener() { @Override public void onSuccess(LogRecordWithDLSN value) { - if (LOG.isDebugEnabled()) { - LOG.debug("{} read record from [{} - {}] of {} : {}", - streamName, context.curStartEntryId.get(), context.curEndEntryId.get(), - metadata, value); - } + log.debug() + .attr("streamName", streamName) + .attr("startEntryId", context.curStartEntryId.get()) + .attr("endEntryId", context.curEndEntryId.get()) + .attr("metadata", metadata) + .attr("value", value) + .log("Read record from entries"); if (null != value) { promise.complete(value); return; @@ -394,9 +401,7 @@ private static void asyncReadRecordFromLogSegment( final long startEntryId) { final long lastAddConfirmed = reader.getLastAddConfirmed(); if (lastAddConfirmed < 0) { - if (LOG.isDebugEnabled()) { - LOG.debug("Log segment {} is empty for {}.", new Object[] { metadata, streamName }); - } + log.debug().attr("metadata", metadata).attr("streamName", streamName).log("Log segment is empty"); promise.complete(null); return; } @@ -429,13 +434,16 @@ private static CompletableFuture asyncReadRecord( new FutureEventListener() { @Override public void onSuccess(final LogSegmentRandomAccessEntryReader reader) { - if (LOG.isDebugEnabled()) { - LOG.debug("{} Opened log segment {} for reading record", streamName, l); - } + log.debug() + .attr("streamName", streamName) + .attr("logSegment", l) + .log("Opened log segment for reading record"); promise.whenComplete((value, cause) -> reader.asyncClose()); - if (LOG.isDebugEnabled()) { - LOG.debug("{} {} scanning {}.", (backward ? "backward" : "forward"), streamName, l); - } + log.debug() + .attr("direction", (backward ? "backward" : "forward")) + .attr("streamName", streamName) + .attr("logSegment", l) + .log("Scanning log segment"); asyncReadRecordFromLogSegment( streamName, reader, l, executorService, scanStartBatchSize, scanMaxBatchSize, diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java index ff23ff07828..973cb74e9f4 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java @@ -26,6 +26,7 @@ import java.util.Set; import java.util.concurrent.CopyOnWriteArraySet; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; @@ -40,10 +41,6 @@ import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.ACL; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * ZooKeeper Client wrapper over {@link org.apache.bookkeeper.zookeeper.ZooKeeperClient}. @@ -58,6 +55,7 @@ *
  • stats about {@link ZKWatcherManager} are exposed under scope watcher_manager * */ +@CustomLog public class ZooKeeperClient { /** @@ -115,8 +113,6 @@ public ZooKeeperConnectionException(String message, Throwable cause) { } } - private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperClient.class.getName()); - private final String name; private final int sessionTimeoutMs; private final int defaultConnectionTimeoutMs; @@ -260,15 +256,20 @@ public void process(WatchedEvent event) { switch (event.getState()) { case Expired: if (null == retryPolicy) { - LOG.info("ZooKeeper {}' session expired. Event: {}", name, event); + log.info() + .attr("name", name) + .attr("event", event) + .log("ZooKeeper' session expired. Event"); closeInternal(); } authenticated = false; break; case Disconnected: if (null == retryPolicy) { - LOG.info("ZooKeeper {} is disconnected from zookeeper now," - + " but it is OK unless we received EXPIRED event.", name); + log.info() + .attr("name", name) + .log("ZooKeeper is disconnected from zookeeper now, but it is OK unless" + + " we received EXPIRED event."); } // Mark as not authenticated if expired or disconnected. In both cases // we lose any attached auth info. Relying on Expired/Disconnected is @@ -287,11 +288,17 @@ public void process(WatchedEvent event) { try { watcher.process(event); } catch (Throwable t) { - LOG.warn("Encountered unexpected exception from watcher {} : ", watcher, t); + log.warn() + .attr("watcher", watcher) + .exception(t) + .log("Encountered unexpected exception from watcher"); } } } catch (Throwable t) { - LOG.warn("Encountered unexpected exception when firing watched event {} : ", event, t); + log.warn() + .attr("event", event) + .exception(t) + .log("Encountered unexpected exception when firing watched event"); } } }; @@ -380,12 +387,12 @@ public boolean unregister(Watcher watcher) { public synchronized void closeInternal() { if (zooKeeper != null) { try { - LOG.info("Closing zookeeper client {}.", name); + log.info().attr("name", name).log("Closing zookeeper client."); zooKeeper.close(); - LOG.info("Closed zookeeper client {}.", name); + log.info().attr("name", name).log("Closed zookeeper client."); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOG.warn("Interrupted trying to close zooKeeper {} : ", name, e); + log.warn().attr("name", name).exception(e).log("Interrupted trying to close zooKeeper"); } finally { zooKeeper = null; } @@ -400,7 +407,7 @@ public synchronized void close() { if (closed) { return; } - LOG.info("Close zookeeper client {}.", name); + log.info().attr("name", name).log("Close zookeeper client."); closeInternal(); // unregister gauges to prevent GC spiral this.watcherManager.unregisterGauges(); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java index 716cbc6e01a..141342e2b9c 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java @@ -21,23 +21,20 @@ import static com.google.common.base.Preconditions.checkNotNull; import java.net.URI; +import lombok.CustomLog; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.zookeeper.RetryPolicy; import org.apache.distributedlog.ZooKeeperClient.Credentials; import org.apache.distributedlog.ZooKeeperClient.DigestCredentials; import org.apache.distributedlog.impl.BKNamespaceDriver; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Builder to build zookeeper client. */ +@CustomLog public class ZooKeeperClientBuilder { - static final Logger LOG = LoggerFactory.getLogger(ZooKeeperClientBuilder.class); - /** * Create a zookeeper client builder to build zookeeper clients. * diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java index 77815fcfaa7..81271d2b979 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java @@ -40,6 +40,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.util.IOUtils; @@ -68,18 +69,13 @@ import org.apache.distributedlog.thrift.AccessControlEntry; import org.apache.distributedlog.tools.DistributedLogTool; import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Admin Tool for DistributedLog. */ +@CustomLog public class DistributedLogAdmin extends DistributedLogTool { - static final Logger LOG = LoggerFactory.getLogger(DistributedLogAdmin.class); - /** * Fix inprogress segment with lower ledger sequence number. * @@ -109,7 +105,7 @@ public static void fixInprogressSegmentWithLowerSequenceNumber(final Namespace n System.out.println(segment.getLogSegmentSequenceNumber() + "\t: " + segment); } } - LOG.info("Get log segments for {} : {}", streamName, segments); + log.info().attr("streamName", streamName).attr("segments", segments).log("Get log segments"); // validate log segments long maxCompletedLogSegmentSequenceNumber = -1L; LogSegmentMetadata inprogressSegment = null; @@ -138,7 +134,11 @@ public static void fixInprogressSegmentWithLowerSequenceNumber(final Namespace n final LogSegmentMetadata newSegment = FutureUtils.result(metadataUpdater.changeSequenceNumber(inprogressSegment, newLogSegmentSequenceNumber)); - LOG.info("Fixed {} : {} -> {} ", streamName, inprogressSegment, newSegment); + log.info() + .attr("streamName", streamName) + .attr("inprogressSegment", inprogressSegment) + .attr("newSegment", newSegment) + .log("Fixed"); if (verbose) { System.out.println("Fixed " + streamName + " : " + inprogressSegment.getZNodeName() + " -> " + newSegment.getZNodeName()); @@ -275,11 +275,11 @@ public void run() { } StreamCandidate candidate; try { - LOG.info("Checking stream {}.", stream); + log.info().attr("stream", stream).log("Checking stream."); candidate = checkStream(namespace, stream, scheduler); - LOG.info("Checked stream {} - {}.", stream, candidate); + log.info().attr("stream", stream).attr("candidate", candidate).log("Checked stream -."); } catch (Throwable e) { - LOG.error("Error on checking stream {} : ", stream, e); + log.error().attr("stream", stream).exception(e).log("Error on checking stream"); doneLatch.countDown(); break; } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java index ab552dc5c78..bf773e820ba 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.net.URI; +import lombok.CustomLog; import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public; import org.apache.bookkeeper.common.annotation.InterfaceStability.Stable; import org.apache.bookkeeper.common.util.OrderedScheduler; @@ -42,8 +43,6 @@ import org.apache.distributedlog.util.ConfUtils; import org.apache.distributedlog.util.DLUtils; import org.apache.distributedlog.util.SimplePermitLimiter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Builder to construct a Namespace. @@ -52,12 +51,11 @@ * @see Namespace * @since 0.3.32 */ +@CustomLog @Public @Stable public class NamespaceBuilder { - private static final Logger logger = LoggerFactory.getLogger(NamespaceBuilder.class); - public static NamespaceBuilder newBuilder() { return new NamespaceBuilder(); } @@ -211,7 +209,7 @@ public Namespace build() FeatureProvider featureProvider; if (null == _featureProvider) { featureProvider = new SettableFeatureProvider("", 0); - logger.info("No feature provider is set. All features are disabled now."); + log.info("No feature provider is set. All features are disabled now."); } else { featureProvider = _featureProvider; } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java index a356cd7db54..c1f6332c185 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java @@ -40,6 +40,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import lombok.CustomLog; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperAccessor; @@ -68,19 +69,13 @@ import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - - /** * DL Auditor will audit DL namespace, e.g. find leaked ledger, report disk usage by streams. */ +@CustomLog public class DLAuditor { - private static final Logger logger = LoggerFactory.getLogger(DLAuditor.class); - private final DistributedLogConfiguration conf; public DLAuditor(DistributedLogConfiguration conf) { @@ -142,7 +137,7 @@ public Pair, Set> collectLedgers(List uris, List collectLedgersFromDL(List uris, List> alloca @Override public void run() { try { - logger.info("Collecting ledgers from {} : {}", uri, aps); + log.info().attr("uri", uri).attr("aps", aps).log("Collecting ledgers"); collectLedgersFromAllocator(uri, namespace, aps, ledgers); synchronized (ledgers) { - logger.info("Collected {} ledgers from allocators for {} : {} ", - ledgers.size(), uri, ledgers); + log.info() + .attr("numLedgers", ledgers.size()) + .attr("uri", uri) + .attr("ledgers", ledgers) + .log("Collected ledgers from allocators"); } collectLedgersFromDL(uri, namespace, ledgers); } catch (IOException e) { numFailures.incrementAndGet(); - logger.info("Error to collect ledgers from DL : ", e); + log.info().exception(e).log("Error to collect ledgers from DL"); } doneLatch.countDown(); } @@ -274,7 +272,7 @@ public void run() { } } catch (InterruptedException e) { Thread.currentThread().interrupt(); - logger.warn("Interrupted on collecting ledgers from DL : ", e); + log.warn().exception(e).log("Interrupted on collecting ledgers from DL"); throw new DLInterruptedException("Interrupted on collecting ledgers from DL : ", e); } } finally { @@ -309,8 +307,7 @@ private void collectLedgersFromAllocator(final URI uri, } } - - logger.info("Collecting ledgers from allocators for {} : {}", uri, poolQueue); + log.info().attr("uri", uri).attr("poolQueue", poolQueue).log("Collecting ledgers from allocators"); executeAction(poolQueue, 10, new Action() { @Override @@ -340,27 +337,34 @@ private void collectLedgersFromPool(String poolPath) ledgers.add(ledgerId); } } catch (NumberFormatException nfe) { - logger.warn("Invalid ledger found in allocator path {} : ", allocatorPath, nfe); + log.warn() + .attr("allocatorPath", allocatorPath) + .exception(nfe) + .log("Invalid ledger found in allocator path"); } } } } }); - logger.info("Collected ledgers from allocators for {}.", uri); + log.info().attr("uri", uri).log("Collected ledgers from allocators"); } private void collectLedgersFromDL(final URI uri, final Namespace namespace, final Set ledgers) throws IOException { - logger.info("Enumerating {} to collect streams.", uri); + log.info().attr("uri", uri).log("Enumerating to collect streams."); Iterator streams = namespace.getLogs(); final LinkedBlockingQueue streamQueue = new LinkedBlockingQueue(); while (streams.hasNext()) { streamQueue.add(streams.next()); } - logger.info("Collected {} streams from uri {} : {}", streamQueue.size(), uri, streams); + log.info() + .attr("numStreams", streamQueue.size()) + .attr("uri", uri) + .attr("streams", streams) + .log("Collected streams from uri"); executeAction(streamQueue, 10, new Action() { @Override @@ -397,7 +401,7 @@ private List collectLedgersFromStream(Namespace namespace, * @throws IOException */ public Map calculateStreamSpaceUsage(final URI uri) throws IOException { - logger.info("Collecting stream space usage for {}.", uri); + log.info().attr("uri", uri).log("Collecting stream space usage"); Namespace namespace = NamespaceBuilder.newBuilder() .conf(conf) .uri(uri) @@ -428,7 +432,10 @@ public void execute(String stream) throws IOException { streamSpaceUsageMap.put(stream, calculateStreamSpaceUsage(namespace, stream)); if (numStreamsCollected.incrementAndGet() % 1000 == 0) { - logger.info("Calculated {} streams from uri {}.", numStreamsCollected.get(), uri); + log.info() + .attr("numStreamsCollected", numStreamsCollected.get()) + .attr("uri", uri) + .log("Calculated streams from uri."); } } }); @@ -450,10 +457,16 @@ private long calculateStreamSpaceUsage(final Namespace namespace, totalBytes += lh.getLength(); lh.close(); } catch (BKException e) { - logger.error("Failed to open ledger {} : ", segment.getLogSegmentId(), e); + log.error() + .attr("logSegmentId", segment.getLogSegmentId()) + .exception(e) + .log("Failed to open ledger"); throw new IOException("Failed to open ledger " + segment.getLogSegmentId(), e); } catch (InterruptedException e) { - logger.warn("Interrupted on opening ledger {} : ", segment.getLogSegmentId(), e); + log.warn() + .attr("logSegmentId", segment.getLogSegmentId()) + .exception(e) + .log("Interrupted on opening ledger"); Thread.currentThread().interrupt(); throw new DLInterruptedException("Interrupted on opening ledger " + segment.getLogSegmentId(), e); } @@ -481,7 +494,7 @@ public long calculateLedgerSpaceUsage(URI uri) throws IOException { ExecutorService executorService = Executors.newCachedThreadPool(); try { BKDLConfig bkdlConfig = resolveBKDLConfig(zkc, uris); - logger.info("Resolved bookkeeper config : {}", bkdlConfig); + log.info().attr("bkdlConfig", bkdlConfig).log("Resolved bookkeeper config"); BookKeeperClient bkc = BookKeeperClientBuilder.newBuilder() .name("DLAuditor-BK") @@ -559,8 +572,11 @@ public void processResult(int rc, String path, Object ctx) { lm.asyncProcessLedgers(collector, finalCb, null, BKException.Code.OK, BKException.Code.ZKException); try { doneFuture.get(); - logger.info("calculated {} ledgers\n\ttotal bytes = {}\n\ttotal entries = {}", - numLedgers.get(), totalBytes.get(), totalEntries.get()); + log.info() + .attr("numLedgers", numLedgers.get()) + .attr("totalBytes", totalBytes.get()) + .attr("totalEntries", totalEntries.get()) + .log("Calculated ledgers"); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new DLInterruptedException("Interrupted on calculating ledger space : ", e); @@ -604,7 +620,10 @@ public void run() { try { action.execute(item); } catch (IOException ioe) { - logger.error("Failed to execute action on item '{}'", item, ioe); + log.error() + .attr("item", item) + .exception(ioe) + .log("Failed to execute action on item ''"); numFailures.incrementAndGet(); failureLatch.countDown(); break; @@ -625,7 +644,7 @@ public void run() { doneLatch.await(); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); - logger.warn("Interrupted on executing action", ie); + log.warn().exception(ie).log("Interrupted on executing action"); throw new DLInterruptedException("Interrupted on executing action", ie); } } finally { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java index 83ee1cf71b7..642b081edda 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java @@ -31,6 +31,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import lombok.CustomLog; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; @@ -47,16 +48,13 @@ import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * LedgerAllocator impl. */ +@CustomLog public class LedgerAllocatorPool implements LedgerAllocator { - private static final Logger logger = LoggerFactory.getLogger(LedgerAllocatorPool.class); - private final DistributedLogConfiguration conf; private final QuorumConfigProvider quorumConfigProvider; private final BookKeeperClient bkc; @@ -131,7 +129,7 @@ private void initializePool() throws IOException { try { allocators = zkc.get().getChildren(poolPath, false); } catch (KeeperException.NoNodeException e) { - logger.info("Allocator Pool {} doesn't exist. Creating it.", poolPath); + log.info().attr("poolPath", poolPath).log("Allocator Pool doesn't exist. Creating it."); ZkUtils.createFullPathOptimistic(zkc.get(), poolPath, new byte[0], zkc.getDefaultACL(), CreateMode.PERSISTENT); allocators = zkc.get().getChildren(poolPath, false); @@ -231,7 +229,10 @@ public void run() { } }, conf.getZKRetryBackoffStartMillis(), TimeUnit.MILLISECONDS); } catch (RejectedExecutionException ree) { - logger.warn("Failed to schedule rescuing ledger allocator {} : ", ledgerAllocator.allocatePath, ree); + log.warn() + .attr("allocatePath", ledgerAllocator.allocatePath) + .exception(ree) + .log("Failed to schedule rescuing ledger allocator"); } } @@ -246,7 +247,7 @@ private void rescueAllocator(final SimpleLedgerAllocator ledgerAllocator) throws oldAllocator = rescueMap.put(ledgerAllocator.allocatePath, ledgerAllocator); } if (oldAllocator != null) { - logger.info("ledger allocator {} is being rescued.", ledgerAllocator.allocatePath); + log.info().attr("allocatePath", ledgerAllocator.allocatePath).log("ledger allocator is being rescued."); return; } try { @@ -258,12 +259,12 @@ public void processResult(int rc, String path, Object ctx, byte[] data, Stat sta if (KeeperException.Code.OK.intValue() == rc) { Versioned allocatorData = new Versioned(data, new LongVersion(stat.getVersion())); - logger.info("Rescuing ledger allocator {}.", path); + log.info().attr("path", path).log("Rescuing ledger allocator."); newAllocator = new SimpleLedgerAllocator(path, allocatorData, quorumConfigProvider, zkc, bkc); newAllocator.start(); - logger.info("Rescued ledger allocator {}.", path); + log.info().attr("path", path).log("Rescued ledger allocator."); } else if (KeeperException.Code.NONODE.intValue() == rc) { - logger.info("Ledger allocator {} doesn't exist, skip rescuing it.", path); + log.info().attr("path", path).log("Ledger allocator doesn't exist, skip rescuing it."); } else { retry = true; } @@ -280,15 +281,20 @@ public void processResult(int rc, String path, Object ctx, byte[] data, Stat sta }, null); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); - logger.warn("Interrupted on rescuing ledger allocator {} : ", ledgerAllocator.allocatePath, ie); + log.warn() + .attr("allocatePath", ledgerAllocator.allocatePath) + .exception(ie) + .log("Interrupted on rescuing ledger allocator"); synchronized (LedgerAllocatorPool.this) { rescueMap.remove(ledgerAllocator.allocatePath); } throw new DLInterruptedException("Interrupted on rescuing ledger allocator " + ledgerAllocator.allocatePath, ie); } catch (IOException ioe) { - logger.warn("Failed to rescue ledger allocator {}, retry rescuing it later : ", - ledgerAllocator.allocatePath, ioe); + log.warn() + .attr("allocatePath", ledgerAllocator.allocatePath) + .exception(ioe) + .log("Failed to rescue ledger allocator, retry rescuing it later"); synchronized (LedgerAllocatorPool.this) { rescueMap.remove(ledgerAllocator.allocatePath); } @@ -350,7 +356,10 @@ public void onFailure(Throwable cause) { try { rescueAllocator(allocator); } catch (IOException ioe) { - logger.info("Failed to rescue allocator {}", allocator.allocatePath, ioe); + log.info() + .attr("allocatePath", allocator.allocatePath) + .exception(ioe) + .log("Failed to rescue allocator"); } tryObtainPromise.completeExceptionally(cause); } @@ -396,7 +405,7 @@ void abortObtain(SimpleLedgerAllocator allocator) { try { rescueAllocator(allocator); } catch (DLInterruptedException e) { - logger.warn("Interrupted on rescuing ledger allocator pool {} : ", poolPath, e); + log.warn().attr("poolPath", poolPath).exception(e).log("Interrupted on rescuing ledger allocator pool"); Thread.currentThread().interrupt(); } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/QuorumConfig.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/QuorumConfig.java index 712024e5c31..5d64bef3e2f 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/QuorumConfig.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/QuorumConfig.java @@ -18,16 +18,14 @@ package org.apache.distributedlog.bk; import com.google.common.base.Objects; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import lombok.CustomLog; /** * Configuration for quorums. */ +@CustomLog public class QuorumConfig { - private static final Logger logger = LoggerFactory.getLogger(QuorumConfig.class); - private final int ensembleSize; private final int writeQuorumSize; private final int ackQuorumSize; @@ -38,15 +36,19 @@ public QuorumConfig(int ensembleSize, this.ensembleSize = ensembleSize; if (this.ensembleSize < writeQuorumSize) { this.writeQuorumSize = this.ensembleSize; - logger.warn("Setting write quorum size {} greater than ensemble size {}", - writeQuorumSize, this.ensembleSize); + log.warn() + .attr("writeQuorumSize", writeQuorumSize) + .attr("ensembleSize", this.ensembleSize) + .log("Setting write quorum size greater than ensemble size"); } else { this.writeQuorumSize = writeQuorumSize; } if (this.writeQuorumSize < ackQuorumSize) { this.ackQuorumSize = this.writeQuorumSize; - logger.warn("Setting write ack quorum size {} greater than write quorum size {}", - ackQuorumSize, this.writeQuorumSize); + log.warn() + .attr("ackQuorumSize", ackQuorumSize) + .attr("writeQuorumSize", this.writeQuorumSize) + .log("Setting write ack quorum size greater than write quorum size"); } else { this.ackQuorumSize = ackQuorumSize; } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java index c300c92c5a2..905008e20f6 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java @@ -24,6 +24,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.function.Function; +import lombok.CustomLog; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; @@ -41,16 +42,13 @@ import org.apache.distributedlog.zk.ZKVersionedSetOp; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Allocator to allocate ledgers. */ +@CustomLog public class SimpleLedgerAllocator implements LedgerAllocator, FutureEventListener, OpListener { - static final Logger LOG = LoggerFactory.getLogger(SimpleLedgerAllocator.class); - enum Phase { ALLOCATING, ALLOCATED, HANDING_OVER, HANDED_OVER, ERROR } @@ -244,7 +242,10 @@ private void initialize(Versioned allocationData) { try { ledgerIdLeftFromPrevAllocation = DLUtils.bytes2LogSegmentId(data); } catch (NumberFormatException nfe) { - LOG.warn("Invalid data found in allocator path {} : ", allocatePath, nfe); + log.warn() + .attr("allocatePath", allocatePath) + .exception(nfe) + .log("Invalid data found in allocator path"); } } @@ -252,8 +253,9 @@ private void initialize(Versioned allocationData) { private synchronized void deleteLedgerLeftFromPreviousAllocationIfNecessary() { if (null != ledgerIdLeftFromPrevAllocation) { - LOG.info("Deleting allocated-but-unused ledger left from previous allocation {}.", - ledgerIdLeftFromPrevAllocation); + log.info() + .attr("ledgerIdLeftFromPrevAllocation", ledgerIdLeftFromPrevAllocation) + .log("Deleting allocated-but-unused ledger left from previous allocation."); deleteLedger(ledgerIdLeftFromPrevAllocation); ledgerIdLeftFromPrevAllocation = null; } @@ -332,8 +334,10 @@ public void onAbort(Throwable t) { listenerToNotify = tryObtainListener; if (t instanceof KeeperException && ((KeeperException) t).code() == KeeperException.Code.BADVERSION) { - LOG.info("Set ledger allocator {} to ERROR state after hit bad version : version = {}", - allocatePath, getVersion()); + log.info() + .attr("allocatePath", allocatePath) + .attr("version", getVersion()) + .log("Set ledger allocator to ERROR state after hit bad version"); setPhase(Phase.ERROR); } else { if (Phase.HANDING_OVER == phase) { @@ -350,8 +354,11 @@ public void onAbort(Throwable t) { private synchronized void setPhase(Phase phase) { this.phase = phase; - LOG.info("Ledger allocator {} moved to phase {} : version = {}.", - allocatePath, phase, version); + log.info() + .attr("allocatePath", allocatePath) + .attr("phase", phase) + .attr("version", version) + .log("Ledger allocator moved to phase"); } private synchronized void allocateLedger() { @@ -361,7 +368,10 @@ private synchronized void allocateLedger() { private synchronized void allocateLedger(LedgerMetadata ledgerMetadata) { // make sure previous allocation is already handed over. if (Phase.HANDED_OVER != phase) { - LOG.error("Trying allocate ledger for {} in phase {}, giving up.", allocatePath, phase); + log.error() + .attr("allocatePath", allocatePath) + .attr("phase", phase) + .log("Trying to allocate ledger in wrong phase, giving up"); return; } setPhase(Phase.ALLOCATING); @@ -400,7 +410,7 @@ public void onSuccess(LedgerHandle lh) { @Override public void onFailure(Throwable cause) { - LOG.error("Error creating ledger for allocating {} : ", allocatePath, cause); + log.error().attr("allocatePath", allocatePath).exception(cause).log("Error creating ledger for allocating"); setPhase(Phase.ERROR); failAllocation(cause); } @@ -412,12 +422,18 @@ private synchronized LongVersion getVersion() { private synchronized void setVersion(LongVersion newVersion) { Version.Occurred occurred = newVersion.compare(version); if (occurred == Version.Occurred.AFTER) { - LOG.info("Ledger allocator for {} moved version from {} to {}.", - allocatePath, version, newVersion); + log.info() + .attr("allocatePath", allocatePath) + .attr("version", version) + .attr("newVersion", newVersion) + .log("Ledger allocator version moved"); version = newVersion; } else { - LOG.warn("Ledger allocator for {} received an old version {}, current version is {}.", - allocatePath, newVersion, version); + log.warn() + .attr("allocatePath", allocatePath) + .attr("newVersion", newVersion) + .attr("version", version) + .log("Ledger allocator received an old version"); } } @@ -440,8 +456,11 @@ public void onSuccess(LongVersion version) { public void onFailure(Throwable cause) { setPhase(Phase.ERROR); deleteLedger(lh.getId()); - LOG.error("Fail mark ledger {} as allocated under {} : ", - lh.getId(), allocatePath, cause); + log.error() + .attr("ledgerId", lh.getId()) + .attr("allocatePath", allocatePath) + .exception(cause) + .log("Failed to mark ledger as allocated"); // fail the allocation since failed to mark it as allocated failAllocation(cause); } @@ -455,8 +474,11 @@ void deleteLedger(final long ledgerId) { } deleteFuture.whenComplete((value, cause) -> { if (null != cause) { - LOG.error("Error deleting ledger {} for ledger allocator {}, retrying : ", - ledgerId, allocatePath, cause); + log.error() + .attr("ledgerId", ledgerId) + .attr("allocatePath", allocatePath) + .exception(cause) + .log("Error deleting ledger for ledger allocator, retrying"); if (!isClosing()) { deleteLedger(ledgerId); } @@ -481,7 +503,7 @@ private CompletableFuture closeInternal(boolean cleanup) { closeFuture = closePromise; } if (!cleanup) { - LOG.info("Abort ledger allocator without cleaning up on {}.", allocatePath); + log.info().attr("allocatePath", allocatePath).log("Abort ledger allocator without cleaning up"); FutureUtils.complete(closePromise, null); return closePromise; } @@ -490,7 +512,7 @@ private CompletableFuture closeInternal(boolean cleanup) { } private void cleanupAndClose(final CompletableFuture closePromise) { - LOG.info("Closing ledger allocator on {}.", allocatePath); + log.info().attr("allocatePath", allocatePath).log("Closing ledger allocator"); final ZKTransaction txn = new ZKTransaction(zkc); // try obtain ledger handle tryObtain(txn, new OpListener() { @@ -508,7 +530,7 @@ public void onAbort(Throwable t) { private void complete() { closePromise.complete(null); - LOG.info("Closed ledger allocator on {}.", allocatePath); + log.info().attr("allocatePath", allocatePath).log("Closed ledger allocator"); } }).whenComplete(new FutureEventListener() { @Override @@ -529,10 +551,9 @@ public void onSuccess(List values) { @Override public void onFailure(Throwable cause) { - if (LOG.isDebugEnabled()) { - LOG.debug("Fail to obtain the allocated ledger handle when closing the allocator : " - , cause); - } + log.debug() + .exception(cause) + .log("Fail to obtain the allocated ledger handle when closing the allocator"); FutureUtils.complete(closePromise, null); } }); @@ -540,10 +561,9 @@ public void onFailure(Throwable cause) { @Override public void onFailure(Throwable cause) { - if (LOG.isDebugEnabled()) { - LOG.debug("Fail to obtain the allocated ledger handle when closing the allocator : " - , cause); - } + log.debug() + .exception(cause) + .log("Fail to obtain the allocated ledger handle when closing the allocator"); FutureUtils.complete(closePromise, null); } }); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java index 8149a649860..9d4f63ac957 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java @@ -28,13 +28,12 @@ import java.util.Optional; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.commons.configuration2.ex.ConfigurationException; import org.apache.distributedlog.DistributedLogConfiguration; import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration; import org.apache.distributedlog.common.config.ConcurrentConstConfiguration; import org.apache.distributedlog.common.config.ConfigurationSubscription; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Encapsulates creation of DynamicDistributedLogConfiguration instances. Ensures one instance per @@ -42,8 +41,8 @@ * Notes: * Once loaded, stays loaded until shutdown. Caller ensures small finite number of configs are created. */ +@CustomLog public class DynamicConfigurationFactory { - private static final Logger LOG = LoggerFactory.getLogger(DynamicConfigurationFactory.class); private final Map dynamicConfigs; private final List subscriptions; @@ -72,7 +71,7 @@ public synchronized Optional getDynamicConfi dynConf, Collections.singletonList(configFile), executorService, reloadPeriod, reloadUnit); subscriptions.add(subscription); dynamicConfigs.put(configPath, dynConf); - LOG.info("Loaded dynamic configuration at {}", configPath); + log.info().attr("configPath", configPath).log("Loaded dynamic configuration"); } return Optional.of(dynamicConfigs.get(configPath)); } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java index fe955d6acdf..eb389a80e7c 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java @@ -20,24 +20,20 @@ import java.io.IOException; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; +import lombok.CustomLog; import org.apache.bookkeeper.feature.CacheableFeatureProvider; import org.apache.bookkeeper.feature.Feature; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.commons.configuration2.ex.ConfigurationException; import org.apache.distributedlog.DistributedLogConfiguration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Decider based feature provider. */ +@CustomLog public abstract class AbstractFeatureProvider extends CacheableFeatureProvider { - private static final Logger logger = LoggerFactory.getLogger(AbstractFeatureProvider.class); - public static FeatureProvider getFeatureProvider(String rootScope, DistributedLogConfiguration conf, StatsLogger statsLogger) diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java index b11d1486dd1..88061e6d142 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java @@ -18,22 +18,19 @@ package org.apache.distributedlog.feature; import java.util.concurrent.ConcurrentMap; +import lombok.CustomLog; import org.apache.bookkeeper.feature.CacheableFeatureProvider; import org.apache.bookkeeper.feature.Feature; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.feature.SettableFeature; import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Feature Provider that load features from configuration. */ +@CustomLog class ConfigurationFeatureProvider extends CacheableFeatureProvider { - private static final Logger logger = LoggerFactory.getLogger(ConfigurationFeatureProvider.class); - static SettableFeature makeFeature(ConcurrentBaseConfiguration featuresConf, ConcurrentMap features, String featureName) { @@ -46,7 +43,10 @@ static SettableFeature makeFeature(ConcurrentBaseConfiguration featuresConf, if (null != oldFeature) { feature = oldFeature; } else { - logger.info("Load feature {}={}", featureName, availability); + log.info() + .attr("feature", featureName) + .attr("availability", availability) + .log("Load feature"); } } return feature; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java index 826613c550a..7524b03efc4 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java @@ -30,6 +30,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.bookkeeper.feature.Feature; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.feature.SettableFeature; @@ -39,17 +40,13 @@ import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration; import org.apache.distributedlog.common.config.ConfigurationListener; import org.apache.distributedlog.common.config.ConfigurationSubscription; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Feature Provider based dynamic configuration. */ +@CustomLog public class DynamicConfigurationFeatureProvider extends AbstractFeatureProvider implements ConfigurationListener { - private static final Logger logger = LoggerFactory.getLogger(DynamicConfigurationFeatureProvider.class); - private final ConcurrentBaseConfiguration featuresConf; private ConfigurationSubscription featuresConfSubscription; @@ -112,7 +109,10 @@ public void onReload(ConcurrentBaseConfiguration conf) { int availability = conf.getInt(featureName, 0); if (availability != feature.getValue().availability()) { feature.getValue().set(availability); - logger.info("Reload feature {}={}", featureName, availability); + log.info() + .attr("feature", featureName) + .attr("availability", availability) + .log("Reload feature"); } } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java index 959eb0f8033..72e66afeaf1 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java @@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import io.github.merlimat.slog.Logger; import io.netty.channel.EventLoopGroup; import io.netty.channel.epoll.EpollEventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; @@ -73,15 +74,13 @@ import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.common.PathUtils; import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Manager for ZooKeeper/BookKeeper based namespace. */ public class BKNamespaceDriver implements NamespaceDriver { - private static Logger LOG = LoggerFactory.getLogger(BKNamespaceDriver.class); + private static Logger LOG = Logger.get(BKNamespaceDriver.class); // register itself static { @@ -216,8 +215,11 @@ public synchronized NamespaceDriver initialize(DistributedLogConfiguration conf, initialized = true; - LOG.info("Initialized BK namespace driver: clientId = {}, regionId = {}, federated = {}.", - clientId, regionId, bkdlConfig.isFederatedNamespace()); + LOG.info() + .attr("clientId", clientId) + .attr("regionId", regionId) + .attr("federatedNamespace", bkdlConfig.isFederatedNamespace()) + .log("Initialized BK namespace driver"); return this; } @@ -256,7 +258,7 @@ static EventLoopGroup getDefaultEventLoopGroup(int numThreads) { try { return new EpollEventLoopGroup(numThreads, threadFactory); } catch (Throwable t) { - LOG.warn("Could not use Netty Epoll event loop for bookie server:", t); + LOG.warn().exception(t).log("Could not use Netty Epoll event loop for bookie server"); return new NioEventLoopGroup(numThreads, threadFactory); } } else { @@ -328,9 +330,11 @@ private void initializeLogStreamMetadataStores() throws IOException { public static String validateAndGetFullLedgerAllocatorPoolPath(DistributedLogConfiguration conf, URI uri) throws IOException { String poolPath = conf.getLedgerAllocatorPoolPath(); - LOG.info("PoolPath is {}", poolPath); + LOG.info().attr("poolPath", poolPath).log("Validating ledger allocator pool path"); if (null == poolPath || !poolPath.startsWith(".") || poolPath.endsWith("/")) { - LOG.error("Invalid ledger allocator pool path specified when enabling ledger allocator pool: {}", poolPath); + LOG.error() + .attr("poolPath", poolPath) + .log("Invalid ledger allocator pool path specified when enabling ledger allocator pool"); throw new IOException("Invalid ledger allocator pool path specified : " + poolPath); } String poolName = conf.getLedgerAllocatorPoolName(); @@ -342,7 +346,9 @@ private void initializeLogStreamMetadataStores() throws IOException { try { PathUtils.validatePath(rootPath); } catch (IllegalArgumentException iae) { - LOG.error("Invalid ledger allocator pool path specified when enabling ledger allocator pool: {}", poolPath); + LOG.error() + .attr("poolPath", poolPath) + .log("Invalid ledger allocator pool path specified when enabling ledger allocator pool"); throw new IOException("Invalid ledger allocator pool path specified : " + poolPath); } return rootPath; @@ -362,8 +368,10 @@ private void initializeOtherResources() throws IOException { if (null != allocator) { allocator.start(); } - LOG.info("Created ledger allocator pool under {} with size {}.", - allocatorPoolPath, conf.getLedgerAllocatorPoolCoreSize()); + LOG.info() + .attr("allocatorPoolPath", allocatorPoolPath) + .attr("ledgerAllocatorPoolCoreSize", conf.getLedgerAllocatorPoolCoreSize()) + .log("Created ledger allocator pool under with size."); } else { allocator = null; } @@ -372,7 +380,7 @@ private void initializeOtherResources() throws IOException { private void checkState() throws IOException { if (closed.get()) { - LOG.error("BK namespace driver {} is already closed", namespace); + LOG.error().attr("namespace", namespace).log("BK namespace driver is already closed"); throw new AlreadyClosedException("BK namespace driver " + namespace + " is already closed"); } } @@ -482,18 +490,22 @@ public AccessControlManager getAccessControlManager() throws IOException { // Build the access control manager if (aclRootPath == null) { accessControlManager = DefaultAccessControlManager.INSTANCE; - LOG.info("Created default access control manager for {}", namespace); + LOG.info().attr("namespace", namespace).log("Created default access control manager"); } else { if (!isReservedStreamName(aclRootPath)) { throw new IOException("Invalid Access Control List Root Path : " + aclRootPath); } String zkRootPath = namespace.getPath() + "/" + aclRootPath; - LOG.info("Creating zk based access control manager @ {} for {}", - zkRootPath, namespace); + LOG.info() + .attr("zkRootPath", zkRootPath) + .attr("namespace", namespace) + .log("Creating zk based access control manager"); accessControlManager = new ZKAccessControlManager(conf, readerZKC, zkRootPath, scheduler); - LOG.info("Created zk based access control manager @ {} for {}", - zkRootPath, namespace); + LOG.info() + .attr("zkRootPath", zkRootPath) + .attr("namespace", namespace) + .log("Created zk based access control manager"); } } return accessControlManager; @@ -553,10 +565,16 @@ public Map enumerateLogsWithMetadataInNamespace() } } catch (InterruptedException ie) { Thread.currentThread().interrupt(); - LOG.error("Interrupted while deleting " + namespaceRootPath, ie); + LOG.error() + .attr("namespaceRootPath", namespaceRootPath) + .exception(ie) + .log("Interrupted while deleting"); throw new IOException("Interrupted while reading " + namespaceRootPath, ie); } catch (KeeperException ke) { - LOG.error("Error reading" + namespaceRootPath + "entry in zookeeper", ke); + LOG.error() + .attr("namespaceRootPath", namespaceRootPath) + .exception(ke) + .log("Error reading entry in zookeeper"); throw new IOException("Error reading" + namespaceRootPath + "entry in zookeeper", ke); } return result; @@ -585,11 +603,15 @@ public static ZooKeeperClientBuilder createZKClientBuilder(String zkcName, .retryPolicy(retryPolicy) .statsLogger(statsLogger) .zkAclId(conf.getZkAclId()); - LOG.info("Created shared zooKeeper client builder {}: zkServers = {}, numRetries = {}, sessionTimeout = {}," - + " retryBackoff = {}, maxRetryBackoff = {}, zkAclId = {}.", zkcName, zkServers, - conf.getZKNumRetries(), conf.getZKSessionTimeoutMilliseconds(), - conf.getZKRetryBackoffStartMillis(), conf.getZKRetryBackoffMaxMillis(), - conf.getZkAclId()); + LOG.info() + .attr("zkcName", zkcName) + .attr("zkServers", zkServers) + .attr("zKNumRetries", conf.getZKNumRetries()) + .attr("zKSessionTimeoutMilliseconds", conf.getZKSessionTimeoutMilliseconds()) + .attr("zKRetryBackoffStartMillis", conf.getZKRetryBackoffStartMillis()) + .attr("zKRetryBackoffMaxMillis", conf.getZKRetryBackoffMaxMillis()) + .attr("zkAclId", conf.getZkAclId()) + .log("Created shared zooKeeper client builder"); return builder; } @@ -610,8 +632,12 @@ private BookKeeperClientBuilder createBKCBuilder(String bkcName, .requestTimer(requestTimer) .featureProvider(featureProviderOptional) .statsLogger(statsLogger); - LOG.info("Created shared client builder {} : zkServers = {}, ledgersPath = {}, numIOThreads = {}", - bkcName, zkServers, ledgersPath, conf.getBKClientNumberIOThreads()); + LOG.info() + .attr("bkcName", bkcName) + .attr("zkServers", zkServers) + .attr("ledgersPath", ledgersPath) + .attr("bKClientNumberIOThreads", conf.getBKClientNumberIOThreads()) + .log("Created shared client builder"); return builder; } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentFilters.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentFilters.java index ccdd3b252ba..fa270dc97c4 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentFilters.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentFilters.java @@ -20,20 +20,16 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import lombok.CustomLog; import org.apache.distributedlog.DistributedLogConstants; import org.apache.distributedlog.logsegment.LogSegmentFilter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Filters based on current zookeeper log segments. */ +@CustomLog public class ZKLogSegmentFilters { - static final Logger LOG = LoggerFactory.getLogger(ZKLogSegmentFilters.class); - /** * Write handler filter should return all inprogress log segments and the last completed log segment. * Because sequence id & ledger sequence number assignment rely on previous log segments. @@ -70,19 +66,23 @@ public Collection filter(Collection fullList) { result.add(s); } } catch (NumberFormatException nfe) { - LOG.warn("Unexpected sequence number in log segment {} :", s, nfe); + log.warn() + .attr("logSegment", s) + .exception(nfe) + .log("Unexpected sequence number in log segment"); result.add(s); } } else { - LOG.error("Unknown log segment name : {}", s); + log.error().attr("logSegment", s).log("Unknown log segment name"); } } if (null != lastCompletedLogSegmentName) { result.add(lastCompletedLogSegmentName); } - if (LOG.isTraceEnabled()) { - LOG.trace("Filtered log segments {} from {}.", result, fullList); - } + log.trace() + .attr("filtered", result) + .attr("fullList", fullList) + .log("Filtered log segments"); return result; } }; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java index 0cda927de1e..73da1a1e828 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java @@ -31,6 +31,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantReadWriteLock; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.versioning.LongVersion; @@ -61,16 +62,13 @@ import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * ZooKeeper based log segment metadata store. */ +@CustomLog public class ZKLogSegmentMetadataStore implements LogSegmentMetadataStore, Watcher, Children2Callback { - private static final Logger logger = LoggerFactory.getLogger(ZKLogSegmentMetadataStore.class); - private static final List EMPTY_LIST = ImmutableList.of(); private static class ReadLogSegmentsTask implements Runnable, FutureEventListener>> { @@ -116,7 +114,9 @@ public void run() { if (null != store.listeners.get(logSegmentsPath)) { store.zkGetLogSegmentNames(logSegmentsPath, store).whenComplete(this); } else { - logger.debug("Log segments listener for {} has been removed.", logSegmentsPath); + log.debug() + .attr("logSegmentsPath", logSegmentsPath) + .log("Log segments listener for has been removed."); } } } @@ -274,7 +274,7 @@ public void deleteLogSegment(Transaction txn, Op deleteOp = Op.delete( segment.getZkPath(), -1); - logger.info("Delete segment : {}", segment); + log.info().attr("segment", segment).log("Delete segment"); txn.addOp(DefaultZKOp.of(deleteOp, new OpListener() { @Override public void onCommit(Void r) { @@ -285,7 +285,7 @@ public void onCommit(Void r) { @Override public void onAbort(Throwable t) { - logger.info("Aborted transaction on deleting segment {}", segment); + log.info().attr("segment", segment).log("Aborted transaction on deleting segment"); KeeperException.Code kc; if (t instanceof KeeperException) { kc = ((KeeperException) t).code(); @@ -412,7 +412,9 @@ public CompletableFuture>> getLogSegmentNames(String logS if (!listeners.containsKey(logSegmentsPath)) { // listener set has been removed, add it back if (null != listeners.putIfAbsent(logSegmentsPath, listenerSet)) { - logger.debug("Listener set is already found for log segments path {}", logSegmentsPath); + log.debug() + .attr("logSegmentsPath", logSegmentsPath) + .log("Listener set is already found for log segments path"); } } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java index 9003c2ffe55..3cb642fbba4 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.net.URI; import java.util.concurrent.CompletableFuture; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; @@ -36,16 +37,13 @@ import org.apache.distributedlog.util.Utils; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * access to ZKMetadata. */ +@CustomLog @SuppressWarnings("deprecation") public class ZKMetadataAccessor implements org.apache.distributedlog.api.MetadataAccessor { - static final Logger LOG = LoggerFactory.getLogger(ZKMetadataAccessor.class); protected final String name; protected CompletableFuture closePromise; protected final URI uri; @@ -100,12 +98,14 @@ public class ZKMetadataAccessor implements org.apache.distributedlog.api.Metadat BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(this.writerZKC, uri); zkServersForReader = bkdlConfig.getDlZkServersForReader(); } catch (IOException e) { - LOG.warn("Error on resolving dl metadata bindings for {} : ", uri, e); + log.warn().attr("uri", uri).exception(e).log("Error on resolving dl metadata bindings"); zkServersForReader = zkServersForWriter; } if (zkServersForReader.equals(zkServersForWriter)) { - LOG.info("Used same zookeeper servers '{}' for both writers and readers for {}.", - zkServersForWriter, name); + log.info() + .attr("zkServers", zkServersForWriter) + .attr("name", name) + .log("Used same zookeeper servers for both writers and readers."); this.readerZKCBuilder = this.writerZKCBuilder; this.ownReaderZKC = false; } else { @@ -154,9 +154,7 @@ public void createOrUpdateMetadata(byte[] metadata) throws IOException { checkClosedOrInError("createOrUpdateMetadata"); String zkPath = getZKPath(); - if (LOG.isDebugEnabled()) { - LOG.debug("Setting application specific metadata on {}", zkPath); - } + log.debug().attr("zkPath", zkPath).log("Setting application specific metadata"); try { Stat currentStat = writerZKC.get().exists(zkPath, false); if (currentStat == null) { @@ -198,9 +196,7 @@ public void deleteMetadata() throws IOException { public byte[] getMetadata() throws IOException { checkClosedOrInError("createOrUpdateMetadata"); String zkPath = getZKPath(); - if (LOG.isDebugEnabled()) { - LOG.debug("Getting application specific metadata from {}", zkPath); - } + log.debug().attr("zkPath", zkPath).log("Getting application specific metadata"); try { Stat currentStat = readerZKC.get().exists(zkPath, false); if (currentStat == null) { @@ -239,7 +235,7 @@ public CompletableFuture asyncClose() { readerZKC.close(); } } catch (Exception e) { - LOG.warn("Exception while closing distributed log manager", e); + log.warn().exception(e).log("Exception while closing distributed log manager"); } FutureUtils.complete(closeFuture, null); return closeFuture; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java index e26230c27f4..d1627643a8c 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java @@ -25,6 +25,7 @@ import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import lombok.CustomLog; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.distributedlog.DistributedLogConfiguration; import org.apache.distributedlog.ZooKeeperClient; @@ -35,19 +36,14 @@ import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Watcher on watching a given namespace. */ +@CustomLog public class ZKNamespaceWatcher extends NamespaceWatcher implements Runnable, Watcher, AsyncCallback.Children2Callback { - private static final Logger logger = LoggerFactory.getLogger(ZKNamespaceWatcher.class); - private final DistributedLogConfiguration conf; private final URI uri; private final ZooKeeperClient zkc; @@ -68,7 +64,11 @@ private void scheduleTask(Runnable r, long ms) { try { scheduler.schedule(r, ms, TimeUnit.MILLISECONDS); } catch (RejectedExecutionException ree) { - logger.error("Task {} scheduled in {} ms is rejected : ", r, ms, ree); + log.error() + .attr("task", r) + .attr("delayMs", ms) + .exception(ree) + .log("Task scheduled is rejected"); } } @@ -77,7 +77,7 @@ public void run() { try { doWatchNamespaceChanges(); } catch (Exception e) { - logger.error("Encountered unknown exception on watching namespace {} ", uri, e); + log.error().attr("uri", uri).exception(e).log("Encountered unknown exception on watching namespace"); } } @@ -95,7 +95,7 @@ private void doWatchNamespaceChanges() { scheduleTask(this, conf.getZKSessionTimeoutMilliseconds()); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - logger.warn("Interrupted on watching namespace changes for {} : ", uri, e); + log.warn().attr("uri", uri).exception(e).log("Interrupted on watching namespace changes"); scheduleTask(this, conf.getZKSessionTimeoutMilliseconds()); } } @@ -103,7 +103,7 @@ private void doWatchNamespaceChanges() { @Override public void processResult(int rc, String path, Object ctx, List children, Stat stat) { if (KeeperException.Code.OK.intValue() == rc) { - logger.info("Received updated logs under {} : {}", uri, children); + log.info().attr("uri", uri).attr("children", children).log("Received updated logs"); List result = new ArrayList(children.size()); for (String s : children) { if (isReservedStreamName(s)) { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java index 5906226407b..63cbb008517 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java @@ -28,6 +28,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.util.ZkUtils; @@ -43,18 +44,13 @@ import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * ZooKeeper Based {@link org.apache.distributedlog.acl.AccessControlManager}. */ +@CustomLog public class ZKAccessControlManager implements AccessControlManager, Watcher { - private static final Logger logger = LoggerFactory.getLogger(ZKAccessControlManager.class); - private static final int ZK_RETRY_BACKOFF_MS = 500; protected final DistributedLogConfiguration conf; @@ -167,8 +163,10 @@ public void processResult(int rc, String path, Object ctx, List children for (String s : streamsRemoved) { ZKAccessControl accessControl = streamEntries.remove(s); if (null != accessControl) { - logger.info("Removed Access Control Entry for stream {} : {}", - s, accessControl.getAccessControlEntry()); + log.info() + .attr("stream", s) + .attr("accessControlEntry", accessControl.getAccessControlEntry()) + .log("Removed Access Control Entry for stream"); } } if (streamsReceived.isEmpty()) { @@ -185,8 +183,10 @@ public void processResult(int rc, String path, Object ctx, List children @Override public void onSuccess(ZKAccessControl accessControl) { streamEntries.put(streamName, accessControl); - logger.info("Added override access control for stream {} : {}", - streamName, accessControl.getAccessControlEntry()); + log.info() + .attr("streamName", streamName) + .attr("accessControlEntry", accessControl.getAccessControlEntry()) + .log("Added override access control for stream"); complete(); } @@ -195,8 +195,11 @@ public void onFailure(Throwable cause) { if (cause instanceof KeeperException.NoNodeException) { streamEntries.remove(streamName); } else if (cause instanceof ZKAccessControl.CorruptedAccessControlException) { - logger.warn("Access control is corrupted for stream {} @ {},skipped it ...", - streamName, zkRootPath, cause); + log.warn() + .attr("streamName", streamName) + .attr("zkRootPath", zkRootPath) + .exception(cause) + .log("Access control is corrupted for stream, skipped it"); streamEntries.remove(streamName); } else { if (1 == numFailures.incrementAndGet()) { @@ -234,9 +237,10 @@ private void fetchDefaultAccessControlEntry(final CompletableFuture() { @Override public void onSuccess(ZKAccessControl accessControl) { - logger.info("Default Access Control will be changed from {} to {}", - ZKAccessControlManager.this.defaultAccessControl, - accessControl); + log.info() + .attr("defaultAccessControl", ZKAccessControlManager.this.defaultAccessControl) + .attr("accessControl", accessControl) + .log("Default Access Control will be changed"); ZKAccessControlManager.this.defaultAccessControl = accessControl; promise.complete(accessControl); } @@ -244,7 +248,9 @@ public void onSuccess(ZKAccessControl accessControl) { @Override public void onFailure(Throwable cause) { if (cause instanceof KeeperException.NoNodeException) { - logger.info("Default Access Control is missing, creating one for {} ...", zkRootPath); + log.info() + .attr("zkRootPath", zkRootPath) + .log("Default Access Control is missing, creating one"); createDefaultAccessControlEntryIfNeeded(promise); } else { promise.completeExceptionally(cause); @@ -270,7 +276,7 @@ private void createDefaultAccessControlEntryIfNeeded(final CompletableFutureNOTE: current federated namespace isn't optimized for deletion/creation. so don't use it in the workloads * that have lots of creations or deletions.

    */ +@CustomLog public class FederatedZKLogMetadataStore extends NamespaceWatcher implements LogMetadataStore, Watcher, Runnable, FutureEventListener> { - private static final Logger logger = LoggerFactory.getLogger(FederatedZKLogMetadataStore.class); - private static final String ZNODE_SUB_NAMESPACES = ".subnamespaces"; private static final String SUB_NAMESPACE_PREFIX = "NS_"; @@ -139,8 +134,10 @@ public void onStreamsChanged(Iterator newLogsIter) { try { oldLogs = FutureUtils.result(logsFuture); } catch (Exception e) { - logger.error("Unexpected exception when getting logs from a satisfied future of {} : ", - uri, e); + log.error() + .attr("uri", uri) + .exception(e) + .log("Unexpected exception when getting logs from a satisfied future"); } logsFuture = new CompletableFuture>(); } @@ -149,8 +146,11 @@ public void onStreamsChanged(Iterator newLogsIter) { for (String logName : newLogs) { URI oldURI = log2Locations.putIfAbsent(logName, uri); if (null != oldURI && !Objects.equal(uri, oldURI)) { - logger.error("Log {} is found duplicated in multiple locations : old location = {}," - + " new location = {}", logName, oldURI, uri); + log.error() + .attr("logName", logName) + .attr("oldURI", oldURI) + .attr("uri", uri) + .log("Log is found duplicated in multiple locations"); duplicatedLogFound.set(true); } } @@ -216,23 +216,24 @@ public FederatedZKLogMetadataStore( SubNamespace subNs = new SubNamespace(uri); if (null == subNamespaces.putIfAbsent(uri, subNs)) { subNs.watch(); - logger.info("Watched sub namespace {}", uri); + log.info().attr("uri", uri).log("Watched sub namespace"); } } - logger.info("Federated ZK LogMetadataStore is initialized for {}", namespace); + log.info().attr("namespace", namespace).log("Federated ZK LogMetadataStore is initialized"); } private void scheduleTask(Runnable r, long ms) { if (duplicatedLogFound.get()) { - logger.error("Scheduler is halted for federated namespace {} as duplicated log found", - namespace); + log.error() + .attr("namespace", namespace) + .log("Scheduler is halted for federated namespace as duplicated log found"); return; } try { scheduler.schedule(r, ms, TimeUnit.MILLISECONDS); } catch (RejectedExecutionException ree) { - logger.error("Task {} scheduled in {} ms is rejected : ", r, ms, ree); + log.error().attr("task", r).attr("delayMs", ms).exception(ree).log("Task scheduling rejected"); } } @@ -327,7 +328,7 @@ public void processResult(int rc, String path, Object ctx, List children subnamespaces.add(getSubNamespaceURI(ns)); } } catch (URISyntaxException use) { - logger.error("Invalid sub namespace uri found : ", use); + log.error().exception(use).log("Invalid sub namespace uri found"); promise.completeExceptionally(new UnexpectedException( "Invalid sub namespace uri found in " + namespace, use)); return; @@ -360,7 +361,7 @@ public void onSuccess(Set uris) { SubNamespace subNs = new SubNamespace(uri); if (null == subNamespaces.putIfAbsent(uri, subNs)) { subNs.watch(); - logger.info("Watched new sub namespace {}.", uri); + log.info().attr("uri", uri).log("Watched new sub namespace."); notifyOnNamespaceChanges(); } } @@ -523,7 +524,7 @@ public void processResult(int rc, String path, Object ctx, String name) { if (Code.OK.intValue() == rc) { try { URI newUri = getSubNamespaceURI(getNamespaceFromZkPath(name)); - logger.info("Created sub namespace {}", newUri); + log.info().attr("newUri", newUri).log("Created sub namespace"); promise.complete(newUri); } catch (UnexpectedException ue) { promise.completeExceptionally(ue); @@ -667,8 +668,11 @@ public void onSuccess(List> fetchResults) { for (Optional fetchResult : fetchResults) { if (result.isPresent()) { if (fetchResult.isPresent()) { - logger.error("Log {} is found in multiple sub namespaces : {} & {}.", - logName, result.get(), fetchResult.get()); + log.error() + .attr("logName", logName) + .attr("existingUri", result.get()) + .attr("duplicateUri", fetchResult.get()) + .log("Log is found in multiple sub namespaces"); duplicatedLogName.compareAndSet(null, logName); duplicatedLogFound.set(true); fetchPromise.completeExceptionally(new UnexpectedException("Log " + logName diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java index b7f70c9c3a0..f148864d5e8 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java @@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import lombok.CustomLog; import org.apache.bookkeeper.client.AsyncCallback; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; @@ -53,16 +54,13 @@ import org.apache.distributedlog.exceptions.ReadCancelledException; import org.apache.distributedlog.injector.AsyncFailureInjector; import org.apache.distributedlog.logsegment.LogSegmentEntryReader; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * BookKeeper ledger based log segment entry reader. */ +@CustomLog public class BKLogSegmentEntryReader implements Runnable, LogSegmentEntryReader, AsyncCallback.OpenCallback { - private static final Logger logger = LoggerFactory.getLogger(BKLogSegmentEntryReader.class); - private class CacheEntry implements Runnable, AsyncCallback.ReadCallback, AsyncCallback.ReadLastConfirmedAndEntryCallback { @@ -455,8 +453,9 @@ public void openComplete(int rc, LedgerHandle lh, Object ctx) { lh.asyncClose(new AsyncCallback.CloseCallback() { @Override public void closeComplete(int rc, LedgerHandle lh, Object ctx) { - logger.debug("Close the open ledger {} since the log segment reader is already closed", - lh.getId()); + log.debug() + .attr("ledgerId", lh.getId()) + .log("Close the open ledger since the log segment reader is already closed"); } }, null); return; @@ -606,7 +605,6 @@ private void prefetchIfNecessary() { } } - private void issueRead(CacheEntry cacheEntry) { if (isClosed()) { return; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java index 170f75b1a39..007aaed717d 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.concurrent.CompletableFuture; +import lombok.CustomLog; import org.apache.bookkeeper.client.AsyncCallback; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BKException.Code; @@ -48,21 +49,16 @@ import org.apache.distributedlog.logsegment.LogSegmentRandomAccessEntryReader; import org.apache.distributedlog.metadata.LogMetadataForWriter; import org.apache.distributedlog.util.Allocator; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * BookKeeper Based Entry Store. */ +@CustomLog public class BKLogSegmentEntryStore implements LogSegmentEntryStore, AsyncCallback.OpenCallback, AsyncCallback.DeleteCallback { - private static final Logger logger = LoggerFactory.getLogger(BKLogSegmentEntryReader.class); - private static class OpenReaderRequest { private final LogSegmentMetadata segment; @@ -137,11 +133,16 @@ public CompletableFuture deleteLogSegment(LogSegmentMetadata public void deleteComplete(int rc, Object ctx) { DeleteLogSegmentRequest deleteRequest = (DeleteLogSegmentRequest) ctx; if (Code.NoSuchLedgerExistsOnMetadataServerException == rc) { - logger.warn("No ledger {} found to delete for {}.", - deleteRequest.segment.getLogSegmentId(), deleteRequest.segment); + log.warn() + .attr("ledgerId", deleteRequest.segment.getLogSegmentId()) + .attr("segment", deleteRequest.segment) + .log("No ledger found to delete."); } else if (BKException.Code.OK != rc) { - logger.error("Couldn't delete ledger {} from bookkeeper for {} : {}", - deleteRequest.segment.getLogSegmentId(), deleteRequest.segment, BKException.getMessage(rc)); + log.error() + .attr("ledgerId", deleteRequest.segment.getLogSegmentId()) + .attr("segment", deleteRequest.segment) + .attr("error", BKException.getMessage(rc)) + .log("Couldn't delete ledger from bookkeeper"); FutureUtils.completeExceptionally(deleteRequest.deletePromise, new BKTransmitException("Couldn't delete log segment " + deleteRequest.segment, rc)); return; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java index b32b3193a06..f149285b97b 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java @@ -25,6 +25,7 @@ import java.net.URI; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import lombok.CustomLog; import org.apache.distributedlog.DistributedLogConfiguration; import org.apache.distributedlog.DistributedLogConstants; import org.apache.distributedlog.ZooKeeperClient; @@ -35,17 +36,13 @@ import org.apache.thrift.protocol.TJSONProtocol; import org.apache.thrift.transport.TMemoryBuffer; import org.apache.thrift.transport.TMemoryInputTransport; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Configurations for BookKeeper based DL. */ +@CustomLog public class BKDLConfig implements DLConfig { - private static final Logger LOG = LoggerFactory.getLogger(BKDLConfig.class); - private static final int BUFFER_SIZE = 4096; private static final ConcurrentMap cachedDLConfigs = new ConcurrentHashMap(); @@ -55,13 +52,14 @@ public static void propagateConfiguration(BKDLConfig bkdlConfig, DistributedLogC dlConf.setFirstLogSegmentSequenceNumber(bkdlConfig.getFirstLogSegmentSeqNo()); if (bkdlConfig.isFederatedNamespace()) { dlConf.setCreateStreamIfNotExists(false); - LOG.info("Disabled createIfNotExists for federated namespace."); + log.info("Disabled createIfNotExists for federated namespace."); } - LOG.info("Propagate BKDLConfig to DLConfig : encodeRegionID = {}," - + " firstLogSegmentSequenceNumber = {}, createStreamIfNotExists = {}, isFederated = {}.", - dlConf.getEncodeRegionIDInLogSegmentMetadata(), - dlConf.getFirstLogSegmentSequenceNumber(), dlConf.getCreateStreamIfNotExists(), - bkdlConfig.isFederatedNamespace()); + log.info() + .attr("encodeRegionIDInLogSegmentMetadata", dlConf.getEncodeRegionIDInLogSegmentMetadata()) + .attr("firstLogSegmentSequenceNumber", dlConf.getFirstLogSegmentSequenceNumber()) + .attr("createStreamIfNotExists", dlConf.getCreateStreamIfNotExists()) + .attr("federatedNamespace", bkdlConfig.isFederatedNamespace()) + .log("Propagate BKDLConfig to DLConfig"); } public static BKDLConfig resolveDLConfig(ZooKeeperClient zkc, URI uri) throws IOException { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java index b2fa3e653f2..e5bcffcb5f0 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java @@ -33,6 +33,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import io.github.merlimat.slog.Logger; import java.io.IOException; import java.net.URI; import java.util.Collections; @@ -91,15 +92,13 @@ import org.apache.zookeeper.common.PathUtils; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * zookeeper based {@link LogStreamMetadataStore}. */ public class ZKLogStreamMetadataStore implements LogStreamMetadataStore { - private static final Logger LOG = LoggerFactory.getLogger(ZKLogStreamMetadataStore.class); + private static final Logger LOG = Logger.get(ZKLogStreamMetadataStore.class); private final String clientId; private final DistributedLogConfiguration conf; @@ -221,7 +220,7 @@ public void processResult(int rc, String path, Object ctx, Stat stat) { } catch (InterruptedException ie) { Thread.currentThread().interrupt(); - LOG.error("Interrupted while reading {}", logSegmentsPath, ie); + LOG.error().attr("logSegmentsPath", logSegmentsPath).exception(ie).log("Interrupted while reading"); promise.completeExceptionally(new DLInterruptedException("Interrupted while checking " + logSegmentsPath, ie)); } catch (ZooKeeperClient.ZooKeeperConnectionException e) { @@ -269,10 +268,10 @@ public void processResult(final int rc, final String path, Object ctx, String na logMetadata.getFullyQualifiedName()))); } else if (KeeperException.Code.OK.intValue() == rc) { FutureUtils.complete(promise, null); - LOG.trace("Created path {}.", path); + LOG.trace().attr("path", path).log("Created path."); } else if (KeeperException.Code.NODEEXISTS.intValue() == rc) { FutureUtils.complete(promise, null); - LOG.trace("Path {} is already existed.", path); + LOG.trace().attr("path", path).log("Path is already existed."); } else if (DistributedLogConstants.ZK_CONNECTION_EXCEPTION_RESULT_CODE == rc) { FutureUtils.completeExceptionally(promise, new ZooKeeperClient.ZooKeeperConnectionException(path)); @@ -512,19 +511,17 @@ public void processResult(int rc, String path, Object ctx, List result promise.completeExceptionally(new LogExistsException("Someone just created log " + logRootPath)); } else { - if (LOG.isDebugEnabled()) { - StringBuilder builder = new StringBuilder(); - for (OpResult result : resultList) { - if (result instanceof OpResult.ErrorResult) { - OpResult.ErrorResult errorResult = (OpResult.ErrorResult) result; - builder.append(errorResult.getErr()).append(","); - } else { - builder.append(0).append(","); - } + StringBuilder builder = new StringBuilder(); + for (OpResult result : resultList) { + if (result instanceof OpResult.ErrorResult) { + OpResult.ErrorResult errorResult = (OpResult.ErrorResult) result; + builder.append(errorResult.getErr()).append(","); + } else { + builder.append(0).append(","); } - String resultCodeList = builder.substring(0, builder.length() - 1); - LOG.debug("Failed to create log, full rc list = {}", resultCodeList); } + String resultCodeList = builder.substring(0, builder.length() - 1); + LOG.debug().attr("resultCodeList", resultCodeList).log("Failed to create log"); promise.completeExceptionally(new ZKException("Failed to create log " + logRootPath, KeeperException.Code.get(rc))); @@ -587,7 +584,11 @@ static CompletableFuture getLog(final URI uri, try { PathUtils.validatePath(logRootPath); } catch (IllegalArgumentException e) { - LOG.error("Illegal path value {} for stream {}", logRootPath, logName, e); + LOG.error() + .attr("logRootPath", logRootPath) + .attr("logName", logName) + .exception(e) + .log("Illegal path value for stream"); return FutureUtils.exception(new InvalidStreamNameException(logName, "Log name is invalid")); } @@ -852,17 +853,18 @@ private CompletableFuture executeRenameTxn(String oldLogPath, zkOps.addAll(createOps); zkOps.addAll(deleteOps); - if (LOG.isDebugEnabled()) { - for (Op op : zkOps) { - if (op instanceof Create) { - Create create = (Create) op; - LOG.debug("op : create {}", create.getPath()); - } else if (op instanceof Delete) { - Delete delete = (Delete) op; - LOG.debug("op : delete {}, record = {}", delete.getPath(), op.toRequestRecord()); - } else { - LOG.debug("op : {}", op); - } + for (Op op : zkOps) { + if (op instanceof Create) { + Create create = (Create) op; + LOG.debug().attr("path", create.getPath()).log("op create"); + } else if (op instanceof Delete) { + Delete delete = (Delete) op; + LOG.debug() + .attr("path", delete.getPath()) + .attr("requestRecord", op.toRequestRecord()) + .log("op delete"); + } else { + LOG.debug().attr("op", op).log("other op"); } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java index ac7d63dfacf..43a5a739704 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java @@ -17,11 +17,11 @@ */ package org.apache.distributedlog.impl.subscription; - import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.distributedlog.DLSN; import org.apache.distributedlog.ZooKeeperClient; @@ -32,15 +32,12 @@ import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * The subscription state store Implementation. */ +@CustomLog public class ZKSubscriptionStateStore implements SubscriptionStateStore { - private static final Logger logger = LoggerFactory.getLogger(ZKSubscriptionStateStore.class); - private final ZooKeeperClient zooKeeperClient; private final String zkPath; private static final AtomicReferenceFieldUpdater lastCommittedPositionUpdater = @@ -72,11 +69,11 @@ public CompletableFuture getLastCommitPosition() { CompletableFuture getLastCommitPositionFromZK() { final CompletableFuture result = new CompletableFuture(); try { - logger.debug("Reading last commit position from path {}", zkPath); + log.debug().attr("zkPath", zkPath).log("Reading last commit position"); zooKeeperClient.get().getData(zkPath, false, new AsyncCallback.DataCallback() { @Override public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) { - logger.debug("Read last commit position from path {}: rc = {}", zkPath, rc); + log.debug().attr("zkPath", zkPath).attr("rc", rc).log("Read last commit position"); if (KeeperException.Code.NONODE.intValue() == rc) { result.complete(DLSN.NonInclusiveLowerBound); } else if (KeeperException.Code.OK.intValue() != rc) { @@ -86,7 +83,7 @@ public void processResult(int rc, String path, Object ctx, byte[] data, Stat sta DLSN dlsn = DLSN.deserialize(new String(data, StandardCharsets.UTF_8)); result.complete(dlsn); } catch (Exception t) { - logger.warn("Invalid last commit position found from path {}", zkPath, t); + log.warn().attr("zkPath", zkPath).exception(t).log("Invalid last commit position found"); // invalid dlsn recorded in subscription state store result.complete(DLSN.NonInclusiveLowerBound); } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/injector/RandomDelayFailureInjector.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/injector/RandomDelayFailureInjector.java index ef64f90aac3..1a63eecf4ea 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/injector/RandomDelayFailureInjector.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/injector/RandomDelayFailureInjector.java @@ -17,18 +17,16 @@ */ package org.apache.distributedlog.injector; +import lombok.CustomLog; import org.apache.distributedlog.config.DynamicDistributedLogConfiguration; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Injector that injects random delays. */ +@CustomLog public class RandomDelayFailureInjector implements FailureInjector { - private static final Logger LOG = LoggerFactory.getLogger(RandomDelayFailureInjector.class); - private final DynamicDistributedLogConfiguration dynConf; public RandomDelayFailureInjector(DynamicDistributedLogConfiguration dynConf) { @@ -55,7 +53,7 @@ public void inject() { } } catch (InterruptedException ex) { Thread.currentThread().interrupt(); - LOG.warn("delay was interrupted ", ex); + log.warn().exception(ex).log("delay was interrupted"); } } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java index 1e07e89d83a..4ec7e64a39f 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java @@ -19,17 +19,16 @@ import static com.google.common.base.Preconditions.checkNotNull; +import lombok.CustomLog; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.distributedlog.exceptions.OverCapacityException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Collect rate limiter implementation, cost(RequestT), overlimit, etc. behavior. */ +@CustomLog public class ComposableRequestLimiter implements RequestLimiter { - protected static final Logger LOG = LoggerFactory.getLogger(ComposableRequestLimiter.class); private final RateLimiter limiter; private final OverlimitFunction overlimitFunction; diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java index 285197c3cb1..dd34d9433a0 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java @@ -20,20 +20,18 @@ import java.io.IOException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.distributedlog.exceptions.DLInterruptedException; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Lock waiter represents the attempt that application tries to lock. */ +@CustomLog public class LockWaiter { - private static final Logger logger = LoggerFactory.getLogger(LockWaiter.class); - private final String lockId; private final String currentOwner; private final CompletableFuture acquireFuture; @@ -89,9 +87,9 @@ public boolean waitForAcquireQuietly() { } catch (DLInterruptedException ie) { Thread.currentThread().interrupt(); } catch (LockTimeoutException lte) { - logger.debug("Timeout on lock acquiring", lte); + log.debug().exception(lte).log("Timeout on lock acquiring"); } catch (IOException e) { - logger.error("Caught exception waiting for lock acquired", e); + log.error().exception(e).log("Caught exception waiting for lock acquired"); } return success; } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java index 3fc32d54c35..108a998c84f 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java @@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; @@ -36,8 +37,6 @@ import org.apache.distributedlog.exceptions.LockingException; import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException; import org.apache.distributedlog.exceptions.UnexpectedException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Distributed lock, using ZooKeeper. @@ -65,10 +64,9 @@ * Other internal lock related stats are also exposed under `lock`. See {@link SessionLock} * for details. */ +@CustomLog public class ZKDistributedLock implements LockListener, DistributedLock { - static final Logger LOG = LoggerFactory.getLogger(ZKDistributedLock.class); - private final SessionLockFactory lockFactory; private final OrderedScheduler lockStateExecutor; private final String lockPath; @@ -181,7 +179,7 @@ void doAsyncAcquireWithSemaphore(final CompletableFuture acqu void doAsyncAcquire(final CompletableFuture acquirePromise, final long lockTimeout) { - LOG.trace("Async Lock Acquire {}", lockPath); + log.trace().attr("lockPath", lockPath).log("Async Lock Acquire"); try { checkLockState(); } catch (IOException ioe) { @@ -202,7 +200,9 @@ void doAsyncAcquire(final CompletableFuture acquirePromise, public void onSuccess(SessionLock lock) { synchronized (ZKDistributedLock.this) { if (closed) { - LOG.info("Skipping tryLocking lock {} since it is already closed", lockPath); + log.info() + .attr("lockPath", lockPath) + .log("Skipping tryLocking lock since it is already closed"); FutureUtils.completeExceptionally(acquirePromise, newLockClosedException()); return; } @@ -234,7 +234,9 @@ void asyncTryLock(SessionLock lock, public void onSuccess(LockWaiter waiter) { synchronized (ZKDistributedLock.this) { if (closed) { - LOG.info("Skipping acquiring lock {} since it is already closed", lockPath); + log.info() + .attr("lockPath", lockPath) + .log("Skipping acquiring lock since it is already closed"); waiter .getAcquireFuture() .completeExceptionally(new LockingException(lockPath, "lock is already closed.")); @@ -260,7 +262,7 @@ void waitForAcquire(final LockWaiter waiter, new FutureEventListener() { @Override public void onSuccess(Boolean acquired) { - LOG.info("{} acquired lock {}", waiter, lockPath); + log.info().attr("waiter", waiter).attr("lockPath", lockPath).log("acquired lock"); if (acquired) { FutureUtils.complete(acquirePromise, ZKDistributedLock.this); } else { @@ -285,7 +287,7 @@ public void onExpired() { reacquireLock(false); } catch (LockingException le) { // should not happen - LOG.error("Locking exception on re-acquiring lock {} : ", lockPath, le); + log.error().attr("lockPath", lockPath).exception(le).log("Locking exception on re-acquiring lock"); } } @@ -487,7 +489,7 @@ private CompletableFuture reacquireLock(boolean throwLockAcqu if (null != lockReacquireFuture) { return lockReacquireFuture; } - LOG.info("reacquiring lock at {}", lockPath); + log.info().attr("lockPath", lockPath).log("Reacquiring lock"); lockReacquireFuture = lockPromise = new CompletableFuture(); lockReacquireFuture.whenComplete(new FutureEventListener() { @Override diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java index a0ce6b5b9f1..11ff7eb4b08 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java @@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; +import io.github.merlimat.slog.Logger; import java.io.IOException; import java.io.UnsupportedEncodingException; import java.net.URLDecoder; @@ -57,8 +58,6 @@ import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * A lock under a given zookeeper session. This is a one-time lock. @@ -125,7 +124,7 @@ */ class ZKSessionLock implements SessionLock { - static final Logger LOG = LoggerFactory.getLogger(ZKSessionLock.class); + static final Logger LOG = Logger.get(ZKSessionLock.class); private static final String LOCK_PATH_PREFIX = "/member_"; private static final String LOCK_PART_SEP = "_"; @@ -192,7 +191,7 @@ enum State { */ static class StateManagement { - static final Logger LOG = LoggerFactory.getLogger(StateManagement.class); + static final Logger LOG = Logger.get(StateManagement.class); private volatile State state; @@ -202,7 +201,11 @@ static class StateManagement { public void transition(State toState) { if (!validTransition(toState)) { - LOG.error("Invalid state transition from {} to {} ", this.state, toState, getStack()); + LOG.error() + .attr("state", this.state) + .attr("toState", toState) + .exception(getStack()) + .log("Invalid state transition"); } this.state = toState; } @@ -394,21 +397,27 @@ public boolean isLockHeld() { protected void executeLockAction(final int lockEpoch, final LockAction func) { lockStateExecutor.executeOrdered(lockPath, () -> { if (getEpoch() == lockEpoch) { - if (LOG.isTraceEnabled()) { - LOG.trace("{} executing lock action '{}' under epoch {} for lock {}", - lockId, func.getActionName(), lockEpoch, lockPath); - } + LOG.trace() + .attr("lockId", lockId) + .attr("actionName", func.getActionName()) + .attr("lockEpoch", lockEpoch) + .attr("lockPath", lockPath) + .log("executing lock action '' under epoch for lock"); func.execute(); - if (LOG.isTraceEnabled()) { - LOG.trace("{} executed lock action '{}' under epoch {} for lock {}", - lockId, func.getActionName(), lockEpoch, lockPath); - } + LOG.trace() + .attr("lockId", lockId) + .attr("actionName", func.getActionName()) + .attr("lockEpoch", lockEpoch) + .attr("lockPath", lockPath) + .log("executed lock action '' under epoch for lock"); } else { - if (LOG.isTraceEnabled()) { - LOG.trace("{} skipped executing lock action '{}' for lock {}," - + " since epoch is changed from {} to {}.", - lockId, func.getActionName(), lockPath, lockEpoch, getEpoch()); - } + LOG.trace() + .attr("lockId", lockId) + .attr("actionName", func.getActionName()) + .attr("lockPath", lockPath) + .attr("lockEpoch", lockEpoch) + .attr("epoch", getEpoch()) + .log("skipped executing lock action, since epoch changed"); } }); } @@ -430,21 +439,27 @@ protected void executeLockAction(final int lockEpoch, lockStateExecutor.executeOrdered(lockPath, () -> { int currentEpoch = getEpoch(); if (currentEpoch == lockEpoch) { - if (LOG.isTraceEnabled()) { - LOG.trace("{} executed lock action '{}' under epoch {} for lock {}", - lockId, func.getActionName(), lockEpoch, lockPath); - } + LOG.trace() + .attr("lockId", lockId) + .attr("actionName", func.getActionName()) + .attr("lockEpoch", lockEpoch) + .attr("lockPath", lockPath) + .log("executed lock action '' under epoch for lock"); func.execute(); - if (LOG.isTraceEnabled()) { - LOG.trace("{} executed lock action '{}' under epoch {} for lock {}", - lockId, func.getActionName(), lockEpoch, lockPath); - } + LOG.trace() + .attr("lockId", lockId) + .attr("actionName", func.getActionName()) + .attr("lockEpoch", lockEpoch) + .attr("lockPath", lockPath) + .log("executed lock action '' under epoch for lock"); } else { - if (LOG.isTraceEnabled()) { - LOG.trace("{} skipped executing lock action '{}' for lock {}," - + " since epoch is changed from {} to {}.", - lockId, func.getActionName(), lockPath, lockEpoch, currentEpoch); - } + LOG.trace() + .attr("lockId", lockId) + .attr("actionName", func.getActionName()) + .attr("lockPath", lockPath) + .attr("lockEpoch", lockEpoch) + .attr("currentEpoch", currentEpoch) + .log("skipped executing lock action, since epoch changed"); promise.completeExceptionally(new EpochChangedException(lockPath, lockEpoch, currentEpoch)); } }); @@ -681,15 +696,19 @@ public void onSuccess(String currentOwner) { public void onFailure(final Throwable lockCause) { // If tryLock failed due to state changed, we don't need to cleanup if (lockCause instanceof LockStateChangedException) { - LOG.info("skipping cleanup for {} at {} after encountering lock " - + "state change exception : ", lockId, lockPath, lockCause); + LOG.info() + .attr("lockId", lockId) + .attr("lockPath", lockPath) + .exception(lockCause) + .log("skipping cleanup after encountering lock state change exception"); result.completeExceptionally(lockCause); return; } - if (LOG.isDebugEnabled()) { - LOG.debug("{} is cleaning up its lock state for {} due to : ", - lockId, lockPath, lockCause); - } + LOG.debug() + .attr("lockId", lockId) + .attr("lockPath", lockPath) + .exception(lockCause) + .log("Cleaning up lock state"); // If we encountered any exception we should cleanup CompletableFuture unlockResult = asyncUnlock(); @@ -764,13 +783,19 @@ public void execute() { } if (null != currentNode) { - LOG.error("Current node for {} overwritten current = {} new = {}", - lockPath, lockId, getLockIdFromPath(currentNode)); + LOG.error() + .attr("lockPath", lockPath) + .attr("lockId", lockId) + .attr("currentLockId", getLockIdFromPath(currentNode)) + .log("Current node overwritten"); } currentNode = name; currentId = getLockIdFromPath(currentNode); - LOG.trace("{} received member id for lock {}", lockId, currentId); + LOG.trace() + .attr("lockId", lockId) + .attr("currentId", currentId) + .log("received member id for lock"); if (lockState.isExpiredOrClosing()) { // Delete node attempt may have come after PREPARING but before create node, @@ -879,9 +904,13 @@ public void unlock() { FutureUtils.result(unlockResult, lockOpTimeout, TimeUnit.MILLISECONDS); } catch (TimeoutException toe) { // This shouldn't happen unless we lose a watch, and may result in a leaked lock. - LOG.error("Timeout unlocking {} owned by {} : ", lockPath, lockId, toe); + LOG.error() + .attr("lockPath", lockPath) + .attr("lockId", lockId) + .exception(toe) + .log("Timeout unlocking"); } catch (Exception e) { - LOG.warn("{} failed to unlock {} : ", lockId, lockPath, e); + LOG.warn().attr("lockId", lockId).attr("lockPath", lockPath).exception(e).log("failed to unlock"); } } @@ -893,10 +922,12 @@ private void claimOwnership(int lockEpoch) { lockContext.clearLockIds(); // add current lock id lockContext.addLockId(lockId); - if (LOG.isDebugEnabled()) { - LOG.debug("Notify lock waiters on {} at {} : watcher epoch {}, lock epoch {}", - lockPath, System.currentTimeMillis(), lockEpoch, getEpoch()); - } + LOG.debug() + .attr("lockPath", lockPath) + .attr("timestamp", System.currentTimeMillis()) + .attr("lockEpoch", lockEpoch) + .attr("epoch", getEpoch()) + .log("Notify lock waiters"); acquireFuture.complete(true); } @@ -916,7 +947,11 @@ private void unlockInternal(final CompletableFuture promise) { return; } - LOG.info("Lock {} for {} is closed from state {}.", lockId, lockPath, lockState.getState()); + LOG.info() + .attr("lockId", lockId) + .attr("lockPath", lockPath) + .attr("state", lockState.getState()) + .log("Lock for is closed from state."); final boolean skipCleanup = lockState.inState(State.INIT) || lockState.inState(State.EXPIRED); @@ -945,7 +980,7 @@ public void onSuccess(Void complete) { public void onFailure(Throwable cause) { // Delete failure is quite serious (causes lock leak) and should be // handled better - LOG.error("lock node delete failed {} {}", lockId, lockPath); + LOG.error().attr("lockId", lockId).attr("lockPath", lockPath).log("lock node delete failed"); promise.complete(null); } }, lockStateExecutor.chooseThread(lockPath)); @@ -962,14 +997,20 @@ private void deleteLockNode(final CompletableFuture promise) { public void processResult(final int rc, final String path, Object ctx) { lockStateExecutor.executeOrdered(lockPath, () -> { if (KeeperException.Code.OK.intValue() == rc) { - LOG.info("Deleted lock node {} for {} successfully.", path, lockId); + LOG.info().attr("path", path).attr("lockId", lockId).log("Deleted lock node for successfully."); } else if (KeeperException.Code.NONODE.intValue() == rc || KeeperException.Code.SESSIONEXPIRED.intValue() == rc) { - LOG.info("Delete node failed. Node already gone for node {} id {}, rc = {}", - path, lockId, KeeperException.Code.get(rc)); + LOG.info() + .attr("path", path) + .attr("lockId", lockId) + .attr("rc", KeeperException.Code.get(rc)) + .log("Delete node failed. Node already gone"); } else { - LOG.error("Failed on deleting lock node {} for {} : {}", - path, lockId, KeeperException.Code.get(rc)); + LOG.error() + .attr("path", path) + .attr("lockId", lockId) + .attr("rc", KeeperException.Code.get(rc)) + .log("Failed on deleting lock node"); } FailpointUtils.checkFailPointNoThrow(FailpointUtils.FailPointName.FP_LockUnlockCleanup); @@ -1036,8 +1077,11 @@ private void handleNodeDelete(int lockEpoch, final WatchedEvent event) { public void execute() { // The lock is either expired or closed if (!lockState.inState(State.WAITING)) { - LOG.info("{} ignore watched node {} deleted event, since lock state has moved to {}.", - lockId, event.getPath(), lockState.getState()); + LOG.info() + .attr("lockId", lockId) + .attr("path", event.getPath()) + .attr("state", lockState.getState()) + .log("ignore watched node deleted event, since lock state has moved"); return; } lockState.transition(State.PREPARED); @@ -1116,7 +1160,7 @@ public void execute() { return; } if (children.isEmpty()) { - LOG.error("Error, member list is empty for lock {}.", lockPath); + LOG.error().attr("lockPath", lockPath).log("Error, member list is empty for lock."); promise.completeExceptionally(new UnexpectedException("Empty member list for lock " + lockPath)); return; } @@ -1125,12 +1169,10 @@ public void execute() { Collections.sort(children, MEMBER_COMPARATOR); final String cid = currentId; final int memberIndex = children.indexOf(cid); - if (LOG.isDebugEnabled()) { - LOG.debug("{} is the number {} member in the list.", cid, memberIndex); - } + LOG.debug().attr("cid", cid).attr("memberIndex", memberIndex).log("is the number member in the list."); // If we hold the lock if (memberIndex == 0) { - LOG.info("{} acquired the lock {}.", cid, lockPath); + LOG.info().attr("cid", cid).attr("lockPath", lockPath).log("acquired the lock."); claimOwnership(lockWatcher.epoch); promise.complete(cid); } else if (memberIndex > 0) { // we are in the member list but we didn't hold the lock @@ -1159,8 +1201,11 @@ public String getActionName() { } }); } else { - LOG.error("Member {} doesn't exist in the members list {} for lock {}.", - cid, children, lockPath); + LOG.error() + .attr("cid", cid) + .attr("children", children) + .attr("lockPath", lockPath) + .log("Member doesn't exist in the members list for lock."); promise.completeExceptionally( new UnexpectedException("Member " + cid + " doesn't exist in member list " + children + " for lock " + lockPath)); @@ -1209,24 +1254,32 @@ public void execute() { // we should watch it and claim ownership shouldWatch = true; shouldClaimOwnership = true; - LOG.info("LockWatcher {} for {} found its previous session {} held lock," - + " watch it to claim ownership.", myNode, lockPath, currentOwner); + LOG.info() + .attr("myNode", myNode) + .attr("lockPath", lockPath) + .attr("currentOwner", currentOwner) + .log("LockWatcher for found its previous session held lock, watch it to claim ownership."); } else if (lockId.compareTo(currentOwner) == 0 && areLockWaitersInSameSession(siblingNode, ownerNode)) { // I found that my sibling is the current owner with same lock id (client id & session id) // It must be left by any race condition from same zookeeper client shouldWatch = true; shouldClaimOwnership = true; - LOG.info("LockWatcher {} for {} found itself {} already held lock at sibling node {}," - + " watch it to claim ownership.", - myNode, lockPath, lockId, siblingNode); + LOG.info() + .attr("myNode", myNode) + .attr("lockPath", lockPath) + .attr("lockId", lockId) + .attr("siblingNode", siblingNode) + .log("LockWatcher for found itself already held lock at sibling node," + + " watch it to claim ownership."); } else { shouldWatch = wait; if (wait) { - if (LOG.isDebugEnabled()) { - LOG.debug("Current LockWatcher for {} with ephemeral node {}, " - + "is waiting for {} to release lock at {}.", - lockPath, myNode, siblingNode, System.currentTimeMillis()); - } + LOG.debug() + .attr("lockPath", lockPath) + .attr("myNode", myNode) + .attr("siblingNode", siblingNode) + .attr("timestamp", System.currentTimeMillis()) + .log("Current LockWatcher is waiting for sibling to release lock"); } shouldClaimOwnership = false; } @@ -1249,8 +1302,11 @@ public void execute() { if (KeeperException.Code.OK.intValue() == rc) { if (shouldClaimOwnership) { // watch owner successfully - LOG.info("LockWatcher {} claimed ownership for {} after set watcher on {}.", - myNode, lockPath, ownerNode); + LOG.info() + .attr("myNode", myNode) + .attr("lockPath", lockPath) + .attr("ownerNode", ownerNode) + .log("LockWatcher claimed ownership after setting watcher"); claimOwnership(lockWatcher.epoch); promise.complete(currentOwner.getLeft()); } else { @@ -1303,17 +1359,25 @@ class LockWatcher implements Watcher { @Override public void process(WatchedEvent event) { - if (LOG.isDebugEnabled()) { - LOG.debug("Received event {} from lock {} at {} : watcher epoch {}, lock epoch {}.", - event, lockPath, System.currentTimeMillis(), epoch, getEpoch()); - } + LOG.debug() + .attr("event", event) + .attr("lockPath", lockPath) + .attr("timestamp", System.currentTimeMillis()) + .attr("epoch", epoch) + .attr("currentEpoch", getEpoch()) + .log("Received event from lock"); if (event.getType() == Watcher.Event.EventType.None) { switch (event.getState()) { case SyncConnected: break; case Expired: - LOG.info("Session {} is expired for lock {} at {} : watcher epoch {}, lock epoch {}.", - lockId.getRight(), lockPath, System.currentTimeMillis(), epoch, getEpoch()); + LOG.info() + .attr("right", lockId.getRight()) + .attr("lockPath", lockPath) + .attr("timestamp", System.currentTimeMillis()) + .attr("epoch", epoch) + .attr("currentEpoch", getEpoch()) + .log("Session is expired for lock"); handleSessionExpired(epoch); break; default: @@ -1323,13 +1387,16 @@ public void process(WatchedEvent event) { // this handles the case where we have aborted a lock and deleted ourselves but still have a // watch on the nextLowestNode. This is a workaround since ZK doesn't support unsub. if (!event.getPath().equals(watchedNode)) { - LOG.warn("{} (watching {}) ignored watched event from {} ", - lockId, watchedNode, event.getPath()); + LOG.warn() + .attr("lockId", lockId) + .attr("watchedNode", watchedNode) + .attr("path", event.getPath()) + .log("(watching) ignored watched event"); return; } handleNodeDelete(epoch, event); } else { - LOG.warn("Unexpected ZK event: {}", event.getType().name()); + LOG.warn().attr("eventType", event.getType().name()).log("Unexpected ZK event"); } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataCache.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataCache.java index a6da02c3ac6..42330a746a9 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataCache.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataCache.java @@ -17,26 +17,22 @@ */ package org.apache.distributedlog.logsegment; - import com.google.common.base.Ticker; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; import com.google.common.cache.RemovalListener; import com.google.common.cache.RemovalNotification; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.distributedlog.DistributedLogConfiguration; import org.apache.distributedlog.LogSegmentMetadata; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Cache the log segment metadata. */ +@CustomLog public class LogSegmentMetadataCache implements RemovalListener { - private static final Logger logger = LoggerFactory.getLogger(LogSegmentMetadataCache.class); - private final Cache cache; private final boolean isCacheEnabled; @@ -52,7 +48,7 @@ public LogSegmentMetadataCache(DistributedLogConfiguration conf, .recordStats() .build(); this.isCacheEnabled = conf.isLogSegmentCacheEnabled(); - logger.info("Log segment cache is enabled = {}", this.isCacheEnabled); + log.info().attr("cacheEnabled", this.isCacheEnabled).log("Log segment cache configured"); } /** @@ -91,9 +87,7 @@ public LogSegmentMetadata get(String path) { @Override public void onRemoval(RemovalNotification notification) { if (notification.wasEvicted()) { - if (logger.isDebugEnabled()) { - logger.debug("Log segment of {} was evicted.", notification.getKey()); - } + log.debug().attr("path", notification.getKey()).log("Log segment was evicted."); } } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java index 59913f22094..0265ae3180e 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java @@ -28,14 +28,11 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import lombok.CustomLog; import org.apache.commons.lang3.tuple.Pair; import org.apache.distributedlog.DistributedLogConstants; import org.apache.distributedlog.LogSegmentMetadata; import org.apache.distributedlog.exceptions.UnexpectedException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Managing log segments in local cache. @@ -47,10 +44,9 @@ * to change if we change the behavior *

    */ +@CustomLog public class PerStreamLogSegmentCache { - static final Logger LOG = LoggerFactory.getLogger(PerStreamLogSegmentCache.class); - protected final String streamName; protected final boolean validateLogSegmentSequenceNumber; protected final Map logSegments = @@ -101,8 +97,11 @@ public List getLogSegments(Comparator co && segment.getVersion() >= LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value && prevSegment.getLogSegmentSequenceNumber() + 1 != segment.getLogSegmentSequenceNumber()) { - LOG.error("{} found ledger sequence number gap between log segment {} and {}", - streamName, prevSegment, segment); + log.error() + .attr("stream", streamName) + .attr("prevSegment", prevSegment) + .attr("segment", segment) + .log("found ledger sequence number gap between log segments"); throw new UnexpectedException(streamName + " found ledger sequence number gap between log segment " + prevSegment.getLogSegmentSequenceNumber() + " and " + segment.getLogSegmentSequenceNumber()); @@ -121,8 +120,11 @@ public List getLogSegments(Comparator co startSequenceId = segment.getStartSequenceId() + segment.getRecordCount(); if (null != prevSegment && prevSegment.supportsSequenceId() && prevSegment.getStartSequenceId() > segment.getStartSequenceId()) { - LOG.warn("{} found decreasing start sequence id in log segment {}, previous is {}", - streamName, segment, prevSegment); + log.warn() + .attr("stream", streamName) + .attr("segment", segment) + .attr("prevSegment", prevSegment) + .log("found decreasing start sequence id in log segment"); } } else { startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID; @@ -158,8 +160,11 @@ public void add(String name, LogSegmentMetadata metadata) { synchronized (logSegments) { if (!logSegments.containsKey(name)) { logSegments.put(name, metadata); - LOG.info("{} added log segment ({} : {}) to cache.", - streamName, name, metadata); + log.info() + .attr("stream", streamName) + .attr("name", name) + .attr("metadata", metadata) + .log("added log segment to cache."); } LogSegmentMetadata oldMetadata = lid2LogSegments.remove(metadata.getLogSegmentId()); if (null == oldMetadata) { @@ -237,13 +242,13 @@ public LogSegmentMetadata remove(String name) { LogSegmentMetadata metadata = logSegments.remove(name); if (null != metadata) { lid2LogSegments.remove(metadata.getLogSegmentId(), metadata); - if (LOG.isDebugEnabled()) { - LOG.debug("Removed log segment ({} : {}) from cache.", name, metadata); - } + log.debug() + .attr("name", name) + .attr("metadata", metadata) + .log("Removed log segment from cache."); } return metadata; } } - } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/TimeBasedRollingPolicy.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/TimeBasedRollingPolicy.java index e8e1647e110..13c4fa0a2e9 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/TimeBasedRollingPolicy.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/logsegment/TimeBasedRollingPolicy.java @@ -17,18 +17,16 @@ */ package org.apache.distributedlog.logsegment; +import lombok.CustomLog; import org.apache.distributedlog.common.util.Sizable; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * TimeBased Policy for rolling. * */ +@CustomLog public class TimeBasedRollingPolicy implements RollingPolicy { - static final Logger LOG = LoggerFactory.getLogger(TimeBasedRollingPolicy.class); - final long rollingIntervalMs; public TimeBasedRollingPolicy(long rollingIntervalMs) { @@ -39,9 +37,11 @@ public TimeBasedRollingPolicy(long rollingIntervalMs) { public boolean shouldRollover(Sizable sizable, long lastRolloverTimeMs) { long elapsedMs = Utils.elapsedMSec(lastRolloverTimeMs); boolean shouldSwitch = elapsedMs > rollingIntervalMs; - if (shouldSwitch && LOG.isDebugEnabled()) { - LOG.debug("Last Finalize Time: {} elapsed time (MSec): {}", lastRolloverTimeMs, - elapsedMs); + if (shouldSwitch) { + log.debug() + .attr("lastRolloverTimeMs", lastRolloverTimeMs) + .attr("elapsedMs", elapsedMs) + .log("Last Finalize Time"); } return shouldSwitch; } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java index 81df6f280d4..0cd90ac1011 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.io.StringReader; import java.net.URI; +import lombok.CustomLog; import org.apache.distributedlog.DistributedLogConfiguration; import org.apache.distributedlog.ZooKeeperClient; import org.apache.distributedlog.ZooKeeperClientBuilder; @@ -31,19 +32,13 @@ import org.apache.distributedlog.util.Utils; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - - /** * Metadata of a given DL instance. */ +@CustomLog public class DLMetadata { - static final Logger LOG = LoggerFactory.getLogger(DLMetadata.class); - static final String LINE_SPLITTER = "\n"; static final String BK_DL_TYPE = "BKDL"; static final int METADATA_FORMAT_VERSION = 1; @@ -89,9 +84,7 @@ public byte[] serialize() { sb.append(metadataFormatVersion).append(LINE_SPLITTER); sb.append(dlType).append(LINE_SPLITTER); sb.append(dlConfig.serialize()); - if (LOG.isDebugEnabled()) { - LOG.debug("Serialized dl metadata {}.", sb); - } + log.debug().attr("metadata", sb).log("Serialized dl metadata."); return sb.toString().getBytes(UTF_8); } @@ -192,9 +185,7 @@ public static void unbind(URI uri) throws IOException { */ public static DLMetadata deserialize(URI uri, byte[] data) throws IOException { String metadata = new String(data, UTF_8); - if (LOG.isDebugEnabled()) { - LOG.debug("Parsing dl metadata {}.", metadata); - } + log.debug().attr("metadata", metadata).log("Parsing dl metadata."); BufferedReader br = new BufferedReader(new StringReader(metadata)); String versionLine = br.readLine(); if (null == versionLine) { diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java index 92060e63e39..68ebd845502 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java @@ -21,23 +21,21 @@ import static com.google.common.base.Preconditions.checkState; import java.util.concurrent.CompletableFuture; +import lombok.CustomLog; import org.apache.distributedlog.DLSN; import org.apache.distributedlog.DistributedLogConfiguration; import org.apache.distributedlog.LogRecordWithDLSN; import org.apache.distributedlog.LogSegmentMetadata; import org.apache.distributedlog.logsegment.LogSegmentMetadataStore; import org.apache.distributedlog.util.Transaction; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** *The implementation is responsible * for updating the metadata. */ +@CustomLog public class LogSegmentMetadataStoreUpdater implements MetadataUpdater { - static final Logger LOG = LoggerFactory.getLogger(LogSegmentMetadataStoreUpdater.class); - public static MetadataUpdater createMetadataUpdater(DistributedLogConfiguration conf, LogSegmentMetadataStore metadataStore) { return new LogSegmentMetadataStoreUpdater(conf, metadataStore); @@ -164,7 +162,10 @@ protected CompletableFuture updateSegmentMetadata(final LogS protected CompletableFuture addNewSegmentAndDeleteOldSegment( final LogSegmentMetadata newSegment, LogSegmentMetadata oldSegment) { - LOG.info("old segment {} new segment {}", oldSegment, newSegment); + log.info() + .attr("oldSegment", oldSegment) + .attr("newSegment", newSegment) + .log("Changing log segment"); Transaction txn = transaction(); addNewSegmentAndDeleteOldSegment(txn, newSegment, oldSegment); return txn.execute().thenApply((value) -> newSegment); diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java index 95ebafae20c..e9515f51316 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java @@ -27,22 +27,18 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import lombok.CustomLog; import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.distributedlog.DistributedLogConstants; import org.apache.distributedlog.impl.BKNamespaceDriver; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * The basic service for managing a set of namespace drivers. */ +@CustomLog public class NamespaceDriverManager { - private static final Logger logger = LoggerFactory.getLogger(NamespaceDriverManager.class); - static class NamespaceDriverInfo { final Class driverClass; @@ -77,7 +73,7 @@ static void initialize() { } loadInitialDrivers(); initialized = true; - logger.info("DistributedLog NamespaceDriverManager initialized"); + log.info("DistributedLog NamespaceDriverManager initialized"); } private static void loadInitialDrivers() { @@ -98,7 +94,7 @@ private static void loadInitialDrivers() { NamespaceDriverInfo driverInfo = new NamespaceDriverInfo(driver.getClass()); drivers.put(driver.getScheme().toLowerCase(), driverInfo); } catch (Exception ex) { - logger.warn("Failed to load namespace driver {} : ", driverClsName, ex); + log.warn().attr("driverClass", driverClsName).exception(ex).log("Failed to load namespace driver"); } } } @@ -126,7 +122,7 @@ public static void registerDriver(String backend, Class domainNameToNetworkLocation = new ConcurrentHashMap(); @@ -66,7 +63,7 @@ public DNSResolver(String hostRegionOverrides) { for (String override : overrides) { String[] parts = override.split(":"); if (parts.length != 2) { - LOG.warn("Incorrect override specified : {}", override); + log.warn().attr("override", override).log("Incorrect override specified"); } else { hostNameToRegion.putIfAbsent(parts[0], parts[1]); } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java index 7ed75c447fd..9424539fbb0 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java @@ -56,6 +56,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; +import lombok.CustomLog; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperAccessor; @@ -104,16 +105,13 @@ import org.apache.distributedlog.metadata.MetadataUpdater; import org.apache.distributedlog.namespace.NamespaceDriver; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** *DistributedLogTool. */ +@CustomLog public class DistributedLogTool extends Tool { - private static final Logger logger = LoggerFactory.getLogger(DistributedLogTool.class); - static final List EMPTY_LIST = Lists.newArrayList(); static int compareByCompletionTime(long time1, long time2) { @@ -1132,7 +1130,10 @@ int countFromStartToEnd(DistributedLogManager dlm) throws Exception { } ++count; if (count % 1000 == 0) { - logger.info("read {} records from {}...", count, getStreamName()); + log.info() + .attr("count", count) + .attr("streamName", getStreamName()) + .log("read records from..."); } preRecord = record; record = reader.readNext(false); @@ -1157,7 +1158,6 @@ protected String getUsage() { } } - /** * Command used to delete a given stream. */ @@ -1770,7 +1770,6 @@ protected int runBKCommand(BKCommandRunner runner) throws Exception { protected abstract int runBKCmd(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception; } - /** * Per Ledger Command, which parse common options for per ledger. e.g. ledger id. */ diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java index 28e83efc712..ccd2ab0459a 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java @@ -19,14 +19,13 @@ import java.io.IOException; import java.util.concurrent.ConcurrentHashMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import lombok.CustomLog; /** * Utils to Failpoint. */ +@CustomLog public class FailpointUtils { - private static final Logger logger = LoggerFactory.getLogger(FailpointUtils.class); /** * enum for FailPointName. @@ -66,7 +65,7 @@ public boolean checkFailPointNoThrow() { try { return checkFailPoint(); } catch (IOException ex) { - logger.error("failpoint action raised unexpected exception"); + log.error("failpoint action raised unexpected exception"); return true; } } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/Utils.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/Utils.java index 36b8346734d..a8afea82694 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/Utils.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/Utils.java @@ -31,7 +31,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import javax.annotation.Nullable; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.versioning.LongVersion; @@ -55,7 +55,7 @@ /** * Basic Utilities. */ -@Slf4j +@CustomLog public class Utils { /** @@ -748,7 +748,10 @@ public static void abort(@Nullable AsyncAbortable abortable, ioResult(abortable.asyncAbort()); } catch (Exception ioe) { if (swallowIOException) { - log.warn("IOException thrown while aborting Abortable {} : ", abortable, ioe); + log.warn() + .attr("abortable", abortable) + .exception(ioe) + .log("IOException thrown while aborting Abortable"); } else { throw ioe; } diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/LimitedPermitManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/LimitedPermitManager.java index 205fd18985c..550249d72e9 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/LimitedPermitManager.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/LimitedPermitManager.java @@ -22,24 +22,20 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import lombok.CustomLog; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.distributedlog.common.util.PermitManager; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Manager to control all the log segments rolling. */ +@CustomLog public class LimitedPermitManager implements PermitManager, Runnable, Watcher { - static final Logger LOG = LoggerFactory.getLogger(LimitedPermitManager.class); - enum PermitState { ALLOWED, DISALLOWED, DISABLED } @@ -127,8 +123,10 @@ public synchronized void releasePermit(Permit permit) { try { executorService.schedule(this, period, timeUnit); } catch (RejectedExecutionException ree) { - LOG.warn("Failed on scheduling releasing permit in given period ({}ms)." - + " Release it immediately : ", timeUnit.toMillis(period), ree); + log.warn() + .attr("periodMs", timeUnit.toMillis(period)) + .exception(ree) + .log("Failed on scheduling releasing permit in given period (ms). Release it immediately"); semaphore.release(); } } @@ -143,7 +141,7 @@ public synchronized boolean disallowObtainPermits(Permit permit) { int epoch = epochUpdater.getAndIncrement(this); if (epoch == ((EpochPermit) permit).getEpoch()) { this.enablePermits = false; - LOG.info("EnablePermits = {}, Epoch = {}.", this.enablePermits, epoch); + log.info().attr("enablePermits", this.enablePermits).attr("epoch", epoch).log("Permits disabled"); return true; } else { return false; @@ -164,7 +162,7 @@ public synchronized boolean allowObtainPermits() { synchronized void forceSetAllowPermits(boolean allowPermits) { int epoch = epochUpdater.getAndIncrement(this); this.enablePermits = allowPermits; - LOG.info("EnablePermits = {}, Epoch = {}.", this.enablePermits, epoch); + log.info().attr("enablePermits", this.enablePermits).attr("epoch", epoch).log("Permits state updated"); } @Override diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/ZKWatcherManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/ZKWatcherManager.java index afd06c2f2a3..af8699e82a0 100644 --- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/ZKWatcherManager.java +++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/ZKWatcherManager.java @@ -22,15 +22,12 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.LongAdder; +import lombok.CustomLog; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.distributedlog.ZooKeeperClient; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Watcher Manager to manage watchers. @@ -40,10 +37,9 @@ *
  • `num_child_watches`: number of paths that are watched for children changes by this watcher manager. * */ +@CustomLog public class ZKWatcherManager implements Watcher { - private static final Logger logger = LoggerFactory.getLogger(ZKWatcherManager.class); - public static Builder newBuilder() { return new Builder(); } @@ -142,8 +138,10 @@ public Watcher registerChildWatcher(String path, Watcher watcher) { allWatchesGauge.increment(); } } else { - logger.warn("Watcher set for path {} has been changed while registering child watcher {}.", - path, watcher); + log.warn() + .attr("path", path) + .attr("watcher", watcher) + .log("Watcher set for path has been changed while registering child watcher."); } } return this; @@ -152,15 +150,20 @@ public Watcher registerChildWatcher(String path, Watcher watcher) { public void unregisterChildWatcher(String path, Watcher watcher, boolean removeFromServer) { Set watchers = childWatches.get(path); if (null == watchers) { - logger.warn("No watchers found on path {} while unregistering child watcher {}.", - path, watcher); + log.warn() + .attr("path", path) + .attr("watcher", watcher) + .log("No watchers found on path while unregistering child watcher."); return; } synchronized (watchers) { if (watchers.remove(watcher)) { allWatchesGauge.decrement(); } else { - logger.warn("Remove a non-registered child watcher {} from path {}", watcher, path); + log.warn() + .attr("watcher", watcher) + .attr("path", path) + .log("Remove a non-registered child watcher from path"); } if (watchers.isEmpty()) { childWatches.remove(path, watchers); @@ -202,7 +205,7 @@ private void handleKeeperStateEvent(WatchedEvent event) { private void handleChildWatchEvent(WatchedEvent event) { String path = event.getPath(); if (null == path) { - logger.warn("Received zookeeper watch event with null path : {}", event); + log.warn().attr("event", event).log("Received zookeeper watch event with null path"); return; } Set watchers = childWatches.get(path); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java index 3e893b3c723..9ee899ff8e9 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java @@ -34,6 +34,7 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -51,15 +52,13 @@ import org.apache.distributedlog.namespace.NamespaceDriver; import org.apache.distributedlog.util.ConfUtils; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Utility class for setting up bookkeeper ensembles * and bringing individual bookies up and down. */ +@CustomLog public class DLMTestUtil { - protected static final Logger LOG = LoggerFactory.getLogger(DLMTestUtil.class); private static final byte[] payloadStatic = repeatString("abc", 512).getBytes(); static String repeatString(String s, int n) { @@ -73,13 +72,13 @@ static String repeatString(String s, int n) { public static Map readLogSegments(ZooKeeperClient zkc, String ledgerPath) throws Exception { List children = zkc.get().getChildren(ledgerPath, false); - LOG.info("Children under {} : {}", ledgerPath, children); + log.info().attr("ledgerPath", ledgerPath).attr("children", children).log("Children under ledger path"); Map segments = new HashMap(children.size()); for (String child : children) { LogSegmentMetadata segment = Utils.ioResult(LogSegmentMetadata.read(zkc, ledgerPath + "/" + child)); - LOG.info("Read segment {} : {}", child, segment); + log.info().attr("child", child).attr("segment", segment).log("Read segment"); segments.put(segment.getLogSegmentSequenceNumber(), segment); } return segments; @@ -468,10 +467,10 @@ public static ServerConfiguration loadTestBkConf() { try { if (null != confUrl) { conf.loadConf(confUrl); - LOG.info("loaded bk_server.conf from resources"); + log.info("loaded bk_server.conf from resources"); } } catch (org.apache.commons.configuration2.ex.ConfigurationException ex) { - LOG.warn("loading conf failed", ex); + log.warn().exception(ex).log("loading conf failed"); } conf.setAllowLoopback(true); return conf; @@ -481,7 +480,8 @@ public static void validateFutureFailed(CompletableFuture future, Class e try { Utils.ioResult(future); } catch (Exception ex) { - LOG.info("Expected: {} Actual: {}", exClass.getName(), ex.getClass().getName()); + log.info().attr("expected", exClass.getName()).attr("actual", ex.getClass().getName()) + .log("Validating future exception"); assertTrue("exceptions types equal", exClass.isInstance(ex)); } } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java index 71004527971..33c7c1cec5f 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java @@ -23,24 +23,19 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.exceptions.LogEmptyException; import org.apache.distributedlog.exceptions.LogNotFoundException; import org.apache.distributedlog.exceptions.LogReadException; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - - /** * Utils for non blocking reads tests. */ +@CustomLog class NonBlockingReadsTestUtil { - static final Logger LOG = LoggerFactory.getLogger(NonBlockingReadsTestUtil.class); - static final long DEFAULT_SEGMENT_SIZE = 1000; static void readNonBlocking(DistributedLogManager dlm, boolean forceStall) throws Exception { @@ -63,12 +58,12 @@ static void readNonBlocking(DistributedLogManager dlm, } catch (LogNotFoundException lnfe) { } catch (LogEmptyException lee) { } catch (IOException ioe) { - LOG.error("Failed to open reader reading from {}", dlm.getStreamName()); + log.error().attr("stream", dlm.getStreamName()).log("Failed to open reader reading from stream"); throw ioe; } } try { - LOG.info("Created reader reading from {}", dlm.getStreamName()); + log.info().attr("stream", dlm.getStreamName()).log("Created reader reading from stream"); if (forceStall) { reader.getReadHandler().disableReadAheadLogSegmentsNotification(); } @@ -134,15 +129,11 @@ static void writeRecordsForNonBlockingReads(DistributedLogConfiguration conf, Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txId++))); TimeUnit.MILLISECONDS.sleep(300); writer.abort(); - if (LOG.isDebugEnabled()) { - LOG.debug("Recovering Segments"); - } + log.debug("Recovering Segments"); BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true); Utils.ioResult(blplm.recoverIncompleteLogSegments()); Utils.ioResult(blplm.asyncClose()); - if (LOG.isDebugEnabled()) { - LOG.debug("Recovered Segments"); - } + log.debug("Recovered Segments"); } else { Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txId++))); writer.closeAndComplete(); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java index e118e289ea5..96adbf42cc6 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java @@ -23,20 +23,18 @@ import static org.junit.Assert.fail; import java.util.Arrays; +import lombok.CustomLog; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.exceptions.EndOfStreamException; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Test Case for {@link AppendOnlyStreamReader}. */ +@CustomLog public class TestAppendOnlyStreamReader extends TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestAppendOnlyStreamReader.class); @Rule public TestName testNames = new TestName(); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java index 2c8f2f7443d..40b54e4eec3 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java @@ -26,6 +26,7 @@ import java.net.URI; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.exceptions.BKTransmitException; import org.apache.distributedlog.exceptions.EndOfStreamException; @@ -35,16 +36,12 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Test Case for {@link AppendOnlyStreamWriter}. */ +@CustomLog public class TestAppendOnlyStreamWriter extends TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestAppendOnlyStreamWriter.class); @Rule public TestName testNames = new TestName(); @@ -332,7 +329,7 @@ long read(DistributedLogManager dlm, long n) throws Exception { offset += read; } } catch (EndOfStreamException ex) { - LOG.info("Caught ex", ex); + log.info().exception(ex).log("Caught ex"); } finally { reader.close(); } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java index 71a945ec657..aad09015b7e 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java @@ -17,7 +17,6 @@ */ package org.apache.distributedlog; - import static org.apache.distributedlog.DLMTestUtil.validateFutureFailed; import static org.apache.distributedlog.DLMTestUtil.validateFutureSucceededAndGetResult; import static org.apache.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE; @@ -30,6 +29,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.exceptions.LogRecordTooLongException; import org.apache.distributedlog.exceptions.WriteCancelledException; @@ -39,19 +39,13 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - - /** * Test cases for bulk writes. */ +@CustomLog public class TestAsyncBulkWrite extends TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestAsyncBulkWrite.class); - @Rule public TestName runtime = new TestName(); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java index 94d4cf6014f..13079719ed4 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java @@ -31,6 +31,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.distributedlog.api.AsyncLogReader; import org.apache.distributedlog.api.DistributedLogManager; @@ -47,17 +48,12 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - - /** * TestAsyncReaderLock. */ +@CustomLog public class TestAsyncReaderLock extends TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestAsyncReaderLock.class); @Rule public TestName runtime = new TestName(); @@ -403,7 +399,7 @@ public boolean done() { @Override public void onSuccess(final AsyncLogReader reader) { - LOG.info("Reader {} is ready to read entries", name); + log.info().attr("name", name).log("Reader is ready to read entries"); executorService.submit(new Runnable() { @Override public void run() { @@ -427,7 +423,7 @@ private void readEntries(AsyncLogReader reader) { @Override public void onFailure(Throwable cause) { - LOG.error("{} failed to open reader", name, cause); + log.error().attr("name", name).exception(cause).log("failed to open reader"); failed = true; latch.countDown(); } @@ -471,14 +467,14 @@ public void testReaderLockMultiReadersScenario() throws Exception { .conf(localConf).uri(uri).clientId(clientId3).build(); DistributedLogManager dlm3 = namespace3.openLog(name); - LOG.info("{} is opening reader on stream {}", clientId1, name); + log.info().attr("clientId", clientId1).attr("stream", name).log("client is opening reader on stream"); CompletableFuture futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN); AsyncLogReader reader1 = Utils.ioResult(futureReader1); - LOG.info("{} opened reader on stream {}", clientId1, name); + log.info().attr("clientId", clientId1).attr("stream", name).log("client opened reader on stream"); - LOG.info("{} is opening reader on stream {}", clientId2, name); + log.info().attr("clientId", clientId2).attr("stream", name).log("client is opening reader on stream"); CompletableFuture futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN); - LOG.info("{} is opening reader on stream {}", clientId3, name); + log.info().attr("clientId", clientId3).attr("stream", name).log("client is opening reader on stream"); CompletableFuture futureReader3 = dlm3.getAsyncLogReaderWithLock(DLSN.InitialDLSN); ExecutorService executorService = Executors.newCachedThreadPool(); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java index 70c9b0dc32c..10507a46e63 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java @@ -41,6 +41,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import lombok.CustomLog; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.api.LedgerMetadata; @@ -78,14 +79,12 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Test Cases for AsyncReaderWriter. */ +@CustomLog public class TestAsyncReaderWriter extends TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestAsyncReaderWriter.class); protected DistributedLogConfiguration testConf; @@ -217,17 +216,14 @@ public void testSimpleAsyncWrite() throws Exception { @Override public void onSuccess(DLSN value) { if (value.getLogSegmentSequenceNo() != currentLogSegmentSeqNo) { - if (LOG.isDebugEnabled()) { - LOG.debug("LogSegmentSequenceNumber: {}, Expected {}", - value.getLogSegmentSequenceNo(), currentLogSegmentSeqNo); - } + log.debug().attr("logSegmentSequenceNumber", value.getLogSegmentSequenceNo()) + .attr("expected", currentLogSegmentSeqNo).log("LogSegmentSequenceNumber mismatch"); errorsFound.set(true); } if (value.getEntryId() != currentEntryId) { - if (LOG.isDebugEnabled()) { - LOG.debug("EntryId: {}, Expected {}", value.getEntryId(), currentEntryId); - } + log.debug().attr("entryId", value.getEntryId()).attr("expected", currentEntryId) + .log("EntryId mismatch"); errorsFound.set(true); } @@ -236,14 +232,14 @@ public void onSuccess(DLSN value) { } syncLatch.countDown(); - if (LOG.isDebugEnabled()) { - LOG.debug("SyncLatch: {}", syncLatch.getCount()); - } + log.debug().attr("syncLatch", syncLatch.getCount()).log("SyncLatch"); } @Override public void onFailure(Throwable cause) { - LOG.error("Encountered exception on writing record {} in log segment {}", - currentEntryId, currentLogSegmentSeqNo); + log.error() + .attr("entryId", currentEntryId) + .attr("logSegmentSequenceNumber", currentLogSegmentSeqNo) + .log("Encountered exception on writing record"); errorsFound.set(true); } }); @@ -355,16 +351,14 @@ public void onSuccess(LogRecordWithDLSN value) { assertTrue(value.getSequenceId() < 0); assertTrue(value.getSequenceId() > startSequenceId); } - LOG.info("Received record {} from {}", value, reader.getStreamName()); + log.info().attr("record", value).attr("stream", reader.getStreamName()).log("Received record"); assertTrue(!value.isControl()); assertTrue(value.getDlsn().getSlotId() == 0); assertTrue(value.getDlsn().compareTo(startPosition) >= 0); DLMTestUtil.verifyLargeLogRecord(value); } catch (Exception exc) { - if (LOG.isDebugEnabled()) { - LOG.debug("Exception Encountered when verifying log record {} : ", - value.getDlsn(), exc); - } + log.debug().attr("dlsn", value.getDlsn()).exception(exc) + .log("Exception Encountered when verifying log record"); errorsFound.set(true); completionLatch.countDown(); return; @@ -385,7 +379,10 @@ public void onSuccess(LogRecordWithDLSN value) { } @Override public void onFailure(Throwable cause) { - LOG.error("Encountered Exception on reading {}", reader.getStreamName(), cause); + log.error() + .attr("stream", reader.getStreamName()) + .exception(cause) + .log("Encountered Exception on reading"); errorsFound.set(true); completionLatch.countDown(); } @@ -473,7 +470,7 @@ public void testBulkAsyncRead() throws Exception { break; } List records = Utils.ioResult(reader.readBulk(20)); - LOG.info("Bulk read {} entries.", records.size()); + log.info().attr("numEntries", records.size()).log("Bulk read entries"); assertTrue(records.size() >= 1); for (LogRecordWithDLSN record : records) { @@ -673,12 +670,14 @@ static class WriteFutureEventListener implements FutureEventListener { @Override public void onSuccess(DLSN value) { if (value.getLogSegmentSequenceNo() != currentLogSegmentSeqNo) { - LOG.error("Ledger Seq No: {}, Expected: {}", value.getLogSegmentSequenceNo(), currentLogSegmentSeqNo); + log.error().attr("ledgerSeqNo", value.getLogSegmentSequenceNo()) + .attr("expected", currentLogSegmentSeqNo).log("Ledger Seq No mismatch"); errorsFound.set(true); } if (verifyEntryId && value.getEntryId() != currentEntryId) { - LOG.error("EntryId: {}, Expected: {}", value.getEntryId(), currentEntryId); + log.error().attr("entryId", value.getEntryId()).attr("expected", currentEntryId) + .log("EntryId mismatch"); errorsFound.set(true); } syncLatch.countDown(); @@ -689,8 +688,8 @@ public void onSuccess(DLSN value) { */ @Override public void onFailure(Throwable cause) { - LOG.error("Encountered failures on writing record as (lid = {}, eid = {}) :", - currentLogSegmentSeqNo, currentEntryId, cause); + log.error().attr("lid", currentLogSegmentSeqNo).attr("eid", currentEntryId).exception(cause) + .log("Encountered failures on writing record"); errorsFound.set(true); syncLatch.countDown(); } @@ -760,7 +759,6 @@ void testSimpleAsyncReadWriteInternal(String name, boolean immediateFlush, dlm.close(); } - /** * Test Case: starting reading when the streams don't exist. * @@ -828,7 +826,6 @@ public void testSimpleAsyncReadWriteStartEmpty() throws Exception { dlm.close(); } - /** * Test Case: starting reading when the streams don't exist. * {@link https://issues.apache.org/jira/browse/DL-42} @@ -1057,7 +1054,7 @@ public void run() { } catch (ReadCancelledException rce) { receiveExpectedException.set(true); } catch (Throwable t) { - LOG.error("Receive unexpected exception on reading stream {} : ", name, t); + log.error().attr("stream", name).exception(t).log("Receive unexpected exception on reading stream"); } readLatch.countDown(); } @@ -1108,9 +1105,7 @@ public void testAsyncWriteWithMinDelayBetweenFlushes() throws Exception { @Override public void onSuccess(DLSN value) { syncLatch.countDown(); - if (LOG.isDebugEnabled()) { - LOG.debug("SyncLatch: {} ; DLSN: {} ", syncLatch.getCount(), value); - } + log.debug().attr("syncLatch", syncLatch.getCount()).attr("dlsn", value).log("SyncLatch"); } @Override public void onFailure(Throwable cause) { @@ -1137,8 +1132,8 @@ public void onFailure(Throwable cause) { assertTrue(success); LogRecordWithDLSN last = dlm.getLastLogRecord(); - LOG.info("Last Entry {}; elapsed time {}", - last.getDlsn().getEntryId(), executionTime.elapsed(TimeUnit.MILLISECONDS)); + log.info().attr("lastEntry", last.getDlsn().getEntryId()) + .attr("elapsedMs", executionTime.elapsed(TimeUnit.MILLISECONDS)).log("Last Entry"); // Regardless of how many records we wrote; the number of BK entries should always be bounded by the min delay. // Since there are two flush processes--data flush and control flush, and since control flush may also end up @@ -1192,9 +1187,7 @@ public void testAsyncWriteWithMinDelayBetweenFlushesFlushFailure() throws Except Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txid++))); fail("should have thrown"); } catch (LockingException ex) { - if (LOG.isDebugEnabled()) { - LOG.debug("caught exception ", ex); - } + log.debug().exception(ex).log("caught exception"); } writer.close(); @@ -1349,7 +1342,8 @@ public void testCloseAndCompleteLogSegmentWhenCloseFailed() throws Exception { fail("Should fail to complete a log segment when its ledger is fenced"); } catch (IOException ioe) { // expected - LOG.error("Failed to close and complete log segment {} : ", logWriter.getFullyQualifiedLogSegment(), ioe); + log.error().attr("logSegment", logWriter.getFullyQualifiedLogSegment()).exception(ioe) + .log("Failed to close and complete log segment"); } List segments = dlm.getLogSegments(); @@ -1643,7 +1637,7 @@ public void testMaxReadAheadRecords() throws Exception { BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN); record = Utils.ioResult(reader.readNext()); - LOG.info("Read record {}", record); + log.info().attr("record", record).log("Read record"); assertEquals(1L, record.getTransactionId()); assertNotNull(reader.getReadAheadReader()); @@ -1651,7 +1645,7 @@ record = Utils.ioResult(reader.readNext()); for (int i = 2; i <= numRecords; i++) { record = Utils.ioResult(reader.readNext()); - LOG.info("Read record {}", record); + log.info().attr("record", record).log("Read record"); assertEquals((long) i, record.getTransactionId()); TimeUnit.MILLISECONDS.sleep(20); int numCachedEntries = reader.getReadAheadReader().getNumCachedEntries(); @@ -2139,7 +2133,7 @@ public void onFailure(Throwable cause) { AsyncLogReader reader2 = readDLM2.getAsyncLogReader(DLSN.InitialDLSN); for (int i = 0; i < 11; i++) { LogRecordWithDLSN record = Utils.ioResult(reader2.readNext()); - LOG.info("Read record {}", record); + log.info().attr("record", record).log("Read record"); if (i < 5) { assertEquals(new DLSN(1L, i, 0L), record.getDlsn()); assertEquals(1L + i, record.getTransactionId()); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java index dcc422e7861..f356e4da98a 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java @@ -37,7 +37,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; @@ -75,15 +75,12 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Test Cases for {@link DistributedLogManager}. */ -@Slf4j +@CustomLog public class TestBKDistributedLogManager extends TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestBKDistributedLogManager.class); private static final Random RAND = new Random(System.currentTimeMillis()); @@ -306,7 +303,7 @@ public void testWriteRestartFrom1() throws Exception { fail("Shouldn't be able to start another journal from " + txid + " when one already exists"); } catch (Exception ioe) { - LOG.info("Caught exception as expected", ioe); + log.info().exception(ioe).log("Caught exception as expected"); } finally { out.close(); } @@ -319,7 +316,7 @@ public void testWriteRestartFrom1() throws Exception { fail("Shouldn't be able to start another journal from " + txid + " when one already exists"); } catch (TransactionIdOutOfOrderException rste) { - LOG.info("Caught exception as expected", rste); + log.info().exception(rste).log("Caught exception as expected"); } finally { out.close(); } @@ -719,7 +716,7 @@ record = reader.readNext(false); record = reader.readNext(false); } } catch (EndOfStreamException exc) { - LOG.info("Encountered EndOfStream on reading records after {}", record); + log.info().attr("record", record).log("Encountered EndOfStream on reading records after record"); exceptionEncountered = true; } assertEquals((txid - 1), numTrans); @@ -1004,17 +1001,17 @@ public void onLogStreamDeleted() { }); long txid = 1; for (int i = 0; i < numSegments; i++) { - LOG.info("Waiting for creating log segment {}.", i); + log.info().attr("segment", i).log("Waiting for creating log segment"); latches[i].await(); - LOG.info("Creating log segment {}.", i); + log.info().attr("segment", i).log("Creating log segment"); BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned(); - LOG.info("Created log segment {}.", i); + log.info().attr("segment", i).log("Created log segment"); for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) { LogRecord op = DLMTestUtil.getLogRecordInstance(txid++); out.write(op); } out.closeAndComplete(); - LOG.info("Completed log segment {}.", i); + log.info().attr("segment", i).log("Completed log segment"); } latches[numSegments].await(); assertEquals(0, numFailures.get()); @@ -1043,13 +1040,13 @@ public void testGetLastDLSN() throws Exception { DistributedLogManager dlm = createNewDLM(confLocal, name); BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned(); long txid = 1; - LOG.info("Writing 10 control records"); + log.info("Writing 10 control records"); for (int i = 0; i < 10; i++) { LogRecord record = DLMTestUtil.getLogRecordInstance(txid++); record.setControl(); Utils.ioResult(writer.writeControlRecord(record)); } - LOG.info("10 control records are written"); + log.info("10 control records are written"); try { dlm.getLastDLSN(); @@ -1059,13 +1056,13 @@ public void testGetLastDLSN() throws Exception { } writer.closeAndComplete(); - LOG.info("Completed first log segment"); + log.info("Completed first log segment"); writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned(); Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txid++))); - LOG.info("Completed second log segment"); + log.info("Completed second log segment"); - LOG.info("Writing another 10 control records"); + log.info("Writing another 10 control records"); for (int i = 1; i < 10; i++) { LogRecord record = DLMTestUtil.getLogRecordInstance(txid++); record.setControl(); @@ -1075,7 +1072,7 @@ public void testGetLastDLSN() throws Exception { assertEquals(new DLSN(2, 0, 0), dlm.getLastDLSN()); writer.closeAndComplete(); - LOG.info("Completed third log segment"); + log.info("Completed third log segment"); assertEquals(new DLSN(2, 0, 0), dlm.getLastDLSN()); @@ -1177,7 +1174,7 @@ public void testTruncationValidation() throws Exception { Map segmentList = DLMTestUtil.readLogSegments(zookeeperClient, LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName())); - LOG.info("Read segments before truncating first segment : {}", segmentList); + log.info().attr("segments", segmentList).log("Read segments before truncating first segment"); MetadataUpdater updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater( confLocal, metadataStore); @@ -1186,7 +1183,7 @@ public void testTruncationValidation() throws Exception { segmentList = DLMTestUtil.readLogSegments(zookeeperClient, LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName())); - LOG.info("Read segments after truncated first segment : {}", segmentList); + log.info().attr("segments", segmentList).log("Read segments after truncated first segment"); { LogReader reader = dlm.getInputStream(DLSN.InitialDLSN); @@ -1209,7 +1206,7 @@ public void testTruncationValidation() throws Exception { segmentList = DLMTestUtil.readLogSegments(zookeeperClient, LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName())); - LOG.info("Read segments after marked first segment as active : {}", segmentList); + log.info().attr("segments", segmentList).log("Read segments after marked first segment as active"); updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, metadataStore); Utils.ioResult(updater.setLogSegmentTruncated(segmentList.get(2L))); @@ -1217,7 +1214,7 @@ public void testTruncationValidation() throws Exception { segmentList = DLMTestUtil.readLogSegments(zookeeperClient, LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName())); - LOG.info("Read segments after truncated second segment : {}", segmentList); + log.info().attr("segments", segmentList).log("Read segments after truncated second segment"); { AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InitialDLSN); @@ -1287,7 +1284,6 @@ public void testTruncationValidation() throws Exception { Utils.close(reader); } - { LogReader reader = dlm.getInputStream(beyondTruncDLSN); LogRecordWithDLSN record = reader.readNext(false); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java index d0372612fd5..38e7c069e86 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java @@ -33,6 +33,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import lombok.CustomLog; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.api.LogReader; import org.apache.distributedlog.api.LogWriter; @@ -55,21 +56,16 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Test Cases for {@link Namespace}. */ +@CustomLog public class TestBKDistributedLogNamespace extends TestDistributedLogBase { @Rule public TestName runtime = new TestName(); - static final Logger LOG = LoggerFactory.getLogger(TestBKDistributedLogNamespace.class); - protected static DistributedLogConfiguration conf = new DistributedLogConfiguration().setLockTimeout(10) .setEnableLedgerAllocatorPool(true).setLedgerAllocatorPoolName("test"); @@ -307,14 +303,14 @@ public void testAclPermsZkAccessConflict() throws Exception { new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); fail("write should have failed due to perms"); } catch (KeeperException.NoAuthException ex) { - LOG.info("caught exception trying to write with no perms", ex); + log.info().exception(ex).log("caught exception trying to write with no perms"); } try { zkc.get().setData(uri.getPath() + "/test-stream", new byte[0], 0); fail("write should have failed due to perms"); } catch (KeeperException.NoAuthException ex) { - LOG.info("caught exception trying to write with no perms", ex); + log.info().exception(ex).log("caught exception trying to write with no perms"); } } @@ -346,10 +342,10 @@ public void testAclModifyPermsDlmConflict() throws Exception { initDlogMeta("/" + runtime.getMethodName(), "not-test-un", streamName); fail("Write should have failed due to perms"); } catch (ZKException ex) { - LOG.info("Caught exception trying to write with no perms", ex); + log.info().exception(ex).log("Caught exception trying to write with no perms"); assertEquals(KeeperException.Code.NOAUTH, ex.getKeeperExceptionCode()); } catch (Exception ex) { - LOG.info("Caught wrong exception trying to write with no perms", ex); + log.info().exception(ex).log("Caught wrong exception trying to write with no perms"); fail("Wrong exception " + ex.getClass().getName() + " expected " + LockingException.class.getName()); } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java index 323a0f59fe1..6d122c31835 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java @@ -28,6 +28,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.distributedlog.api.AsyncLogWriter; import org.apache.distributedlog.api.DistributedLogManager; @@ -39,17 +40,13 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Test {@link BKLogReadHandler}. */ +@CustomLog public class TestBKLogReadHandler extends TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestBKLogReadHandler.class); - @Rule public TestName runtime = new TestName(); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java index 80abd934113..3d2519b00a0 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java @@ -29,6 +29,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; +import lombok.CustomLog; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; @@ -65,6 +66,7 @@ /** * Test Case for BookKeeper Based Log Segment Writer. */ +@CustomLog public class TestBKLogSegmentWriter extends TestDistributedLogBase { @Rule @@ -537,7 +539,7 @@ public void testAbortShouldFailAllWrites() throws Exception { deferLatch.await(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOG.warn("Interrupted on deferring completion : ", e); + log.warn().exception(e).log("Interrupted on deferring completion"); } }); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java index c2a631e7eb2..5a2b6b0fe65 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java @@ -26,6 +26,7 @@ import java.nio.charset.StandardCharsets; import java.util.HashSet; import java.util.Set; +import lombok.CustomLog; import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.distributedlog.api.AsyncLogWriter; import org.apache.distributedlog.api.DistributedLogManager; @@ -45,6 +46,7 @@ /** * Test {@link BKLogWriteHandler}. */ +@CustomLog public class TestBKLogWriteHandler extends TestDistributedLogBase { @Rule @@ -114,19 +116,19 @@ public void testLedgerNumber() throws Exception { BookKeeperAdmin admin = new BookKeeperAdmin(zkServers); Set s1 = getLedgers(admin); - LOG.info("Ledgers after init: " + s1); + log.info("Ledgers after init: " + s1); LogWriter writer = dlm.openLogWriter(); writer.write(new LogRecord(1, "test-data".getBytes(StandardCharsets.UTF_8))); writer.close(); Set s2 = getLedgers(admin); - LOG.info("Ledgers after write: " + s2); + log.info("Ledgers after write: " + s2); dlm.delete(); assertEquals(1, s2.size() - s1.size()); // exact 1 ledger created only Set s3 = getLedgers(admin); - LOG.info("Ledgers after delete: " + s3); + log.info("Ledgers after delete: " + s3); assertEquals(s1.size(), s3.size()); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java index fc1f70b966a..bf2e9f61c6f 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java @@ -28,24 +28,20 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import lombok.CustomLog; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.api.LogReader; import org.apache.distributedlog.exceptions.LogNotFoundException; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Test Sync Log Reader. */ +@CustomLog public class TestBKSyncLogReader extends TestDistributedLogBase { - private static final Logger logger = LoggerFactory.getLogger(TestBKSyncLogReader.class); - @Rule public TestName testName = new TestName(); @@ -162,7 +158,7 @@ public void testReadRecordsAfterReadAheadCaughtUp() throws Exception { out.flush(); out.commit(); - logger.info("Write first 10 records"); + log.info("Write first 10 records"); // all 10 records are added to the stream // then open a reader to read @@ -173,7 +169,7 @@ public void testReadRecordsAfterReadAheadCaughtUp() throws Exception { TimeUnit.MILLISECONDS.sleep(20); } - logger.info("ReadAhead is caught up with first 10 records"); + log.info("ReadAhead is caught up with first 10 records"); for (long i = 11L; i <= 20L; i++) { LogRecord record = DLMTestUtil.getLogRecordInstance(i); @@ -182,7 +178,7 @@ public void testReadRecordsAfterReadAheadCaughtUp() throws Exception { out.flush(); out.commit(); - logger.info("Write another 10 records"); + log.info("Write another 10 records"); // resume reading from sync reader util it consumes 20 records long expectedTxId = 1L; @@ -223,7 +219,7 @@ public void testReadRecordsWhenReadAheadCatchingUp() throws Exception { out.flush(); out.commit(); - logger.info("Write first 10 records"); + log.info("Write first 10 records"); // open a reader to read BKSyncLogReader reader = (BKSyncLogReader) dlm.getInputStream(1L); @@ -265,7 +261,7 @@ public void testReadRecordsWhenReadAheadCatchingUp2() throws Exception { out.commit(); final AtomicLong nextTxId = new AtomicLong(11L); - logger.info("Write first 10 records"); + log.info("Write first 10 records"); ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor(); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestCancelledRead.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestCancelledRead.java index ae306b87cce..560884cc3f8 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestCancelledRead.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestCancelledRead.java @@ -20,16 +20,15 @@ import static org.junit.Assert.assertNotNull; import java.util.concurrent.CompletableFuture; +import lombok.CustomLog; import org.apache.distributedlog.api.LogWriter; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Test Cases for RollLogSegments. */ +@CustomLog public class TestCancelledRead extends TestDistributedLogBase { - private static final Logger logger = LoggerFactory.getLogger(TestRollLogSegments.class); @Test(timeout = 600000) public void testWritingAndTailing() throws Exception { diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java index 73d7d9df7c1..70a2c4ef5bd 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java @@ -18,20 +18,19 @@ package org.apache.distributedlog; import java.io.File; +import lombok.CustomLog; import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim; import org.apache.bookkeeper.util.IOUtils; import org.apache.commons.lang3.tuple.Pair; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Test Case for {@link LocalDLMEmulator}. */ +@CustomLog public class TestDLMTestUtil { - static final Logger LOG = LoggerFactory.getLogger(TestDLMTestUtil.class); static { // org.apache.zookeeper.test.ClientBase uses FourLetterWordMain, from 3.5.3 four letter words diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java index d1170701e14..6c7be269942 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java @@ -28,6 +28,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; @@ -68,15 +69,12 @@ import org.junit.jupiter.api.TestInfo; import org.junit.rules.TestName; import org.junit.rules.Timeout; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * DistributedLogBase providing test environment setup for other Test Cases. */ +@CustomLog public class TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestDistributedLogBase.class); @Rule public final TestName runtime = new TestName(); @@ -150,7 +148,7 @@ protected static void setupCluster(int nBookies) throws Exception { Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { @Override public void uncaughtException(Thread t, Throwable e) { - LOG.warn("Uncaught exception at Thread {} : ", t.getName(), e); + log.warn().attr("thread", t.getName()).exception(e).log("Uncaught exception"); } }); } @@ -171,7 +169,8 @@ public void setup() throws Exception { try { zkc = LocalDLMEmulator.connectZooKeeper("127.0.0.1", zkPort); } catch (Exception ex) { - LOG.error("hit exception connecting to zookeeper at {}:{}", "127.0.0.1", zkPort, ex); + log.error().attr("host", "127.0.0.1").attr("port", zkPort).exception(ex) + .log("hit exception connecting to zookeeper"); throw ex; } } @@ -230,12 +229,12 @@ public BKDistributedLogManager createNewDLM(DistributedLogConfiguration conf, AsyncCloseable resourcesCloseable = new AsyncCloseable() { @Override public CompletableFuture asyncClose() { - LOG.info("Shutting down the scheduler"); + log.info("Shutting down the scheduler"); SchedulerUtils.shutdownScheduler(scheduler, 1, TimeUnit.SECONDS); - LOG.info("Shut down the scheduler"); - LOG.info("Closing the namespace"); + log.info("Shut down the scheduler"); + log.info("Closing the namespace"); namespace.close(); - LOG.info("Closed the namespace"); + log.info("Closed the namespace"); return FutureUtils.Void(); } }; diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java index 059b06511f3..47da10c67b2 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java @@ -20,20 +20,17 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import lombok.CustomLog; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.api.LogReader; import org.apache.distributedlog.util.Utils; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Test Cases for InterleavedReaders. */ +@CustomLog public class TestInterleavedReaders extends TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestInterleavedReaders.class); static { conf.setOutputBufferSize(0); @@ -54,7 +51,7 @@ record = reader0.readNext(false); DLMTestUtil.verifyLogRecord(record); numTrans++; i++; - LOG.info("Read record {}", record); + log.info().attr("record", record).log("Read record"); } } i = 0; @@ -65,7 +62,7 @@ record = reader1.readNext(false); DLMTestUtil.verifyLogRecord(record); numTrans++; i++; - LOG.info("Read record {}", record); + log.info().attr("record", record).log("Read record"); } } return numTrans; @@ -138,7 +135,7 @@ public void testInterleavedReadersWithRollingEdge() throws Exception { } Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid - 1))); Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid - 1))); - LOG.info("Completed {} write", j); + log.info().attr("j", j).log("Completed write"); if (null == reader0) { reader0 = dlmreader0.getInputStream(1); } @@ -220,9 +217,9 @@ public void testInterleavedReadersWithCleanup() throws Exception { writer1.overRideMinTimeStampToKeep(retentionPeriodOverride); } DLSN dlsn1 = Utils.ioResult(writer1.write(DLMTestUtil.getLogRecordInstance(txid++))); - LOG.info("writer1 write record {}", dlsn1); + log.info().attr("dlsn", dlsn1).log("writer1 write record"); DLSN dlsn0 = Utils.ioResult(writer0.write(DLMTestUtil.getLogRecordInstance(txid++))); - LOG.info("writer0 write record {}", dlsn0); + log.info().attr("dlsn", dlsn0).log("writer0 write record"); if (k == 5) { writer0.setForceRolling(false); writer1.setForceRolling(false); @@ -272,9 +269,9 @@ public void testInterleavedReadersWithRecovery() throws Exception { writer1.setForceRecovery(true); } DLSN dlsn1 = Utils.ioResult(writer1.write(DLMTestUtil.getLogRecordInstance(txid++))); - LOG.info("writer1 write record {} - txid = {}", dlsn1, txid - 1); + log.info().attr("dlsn", dlsn1).attr("txid", txid - 1).log("writer1 write record"); DLSN dlsn0 = Utils.ioResult(writer0.write(DLMTestUtil.getLogRecordInstance(txid++))); - LOG.info("writer0 write record {} - txid = {}", dlsn0, txid - 1); + log.info().attr("dlsn", dlsn0).attr("txid", txid - 1).log("writer0 write record"); writer0.setForceRecovery(false); writer1.setForceRecovery(false); } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java index 506e2539140..ed8c9e3195d 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import io.github.merlimat.slog.Logger; import java.net.URI; import java.util.List; import org.apache.distributedlog.api.DistributedLogManager; @@ -29,8 +30,6 @@ import org.apache.distributedlog.api.namespace.NamespaceBuilder; import org.apache.distributedlog.util.Utils; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; @@ -39,7 +38,7 @@ */ public class TestLogSegmentCreation extends TestDistributedLogBase { - static Logger LOG = LoggerFactory.getLogger(TestLogSegmentCreation.class); + static Logger LOG = Logger.get(TestLogSegmentCreation.class); @Test(timeout = 60000) public void testCreateLogSegmentAfterLoseLock() throws Exception { @@ -62,7 +61,7 @@ public void testCreateLogSegmentAfterLoseLock() throws Exception { } List segments = dlm.getLogSegments(); - LOG.info("Segments : {}", segments); + LOG.info().attr("segments", segments).log("Segments"); assertEquals(3, segments.size()); final DistributedLogManager dlm1 = namespace.openLog(name); @@ -79,7 +78,7 @@ public void testCreateLogSegmentAfterLoseLock() throws Exception { Utils.ioResult(writer1.write(DLMTestUtil.getLogRecordInstance(numSegments + 1))); fail("Should fail on writing new log records."); } catch (Throwable t) { - LOG.error("Failed to write entry : ", t); + LOG.error().exception(t).log("Failed to write entry"); } segments = dlm.getLogSegments(); @@ -102,8 +101,8 @@ public void testCreateLogSegmentAfterLoseLock() throws Exception { assertFalse(hasInprogress); assertFalse(hasDuplicatedSegment); - LOG.info("Segments : duplicated = {}, inprogress = {}, {}", - hasDuplicatedSegment, hasInprogress, segments); + LOG.info().attr("duplicated", hasDuplicatedSegment).attr("inprogress", hasInprogress) + .attr("segments", segments).log("Segments"); dlm1.close(); dlm2.close(); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java index 87b2b92b640..158305a1b52 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java @@ -17,13 +17,13 @@ */ package org.apache.distributedlog; - import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; +import lombok.CustomLog; import org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataBuilder; import org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion; import org.apache.distributedlog.LogSegmentMetadata.TruncationStatus; @@ -32,18 +32,13 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Test {@link LogSegmentMetadata}. */ +@CustomLog public class TestLogSegmentMetadata extends ZooKeeperClusterTestCase { - static final Logger LOG = LoggerFactory.getLogger(TestLogSegmentMetadata.class); - static final int TEST_REGION_ID = 0xf - 1; private ZooKeeperClient zkc; @@ -98,7 +93,6 @@ public void testReadMetadataCrossVersionFailure() throws Exception { } } - @Test(timeout = 60000) public void testMutateTruncationStatus() { LogSegmentMetadata metadata = diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java index 5e1688f7c69..8d2a0ce9f5d 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; +import io.github.merlimat.slog.Logger; import java.io.IOException; import java.net.URI; import java.util.List; @@ -37,15 +38,13 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Test Cases for LogSegmentsZK. */ public class TestLogSegmentsZK extends TestDistributedLogBase { - static Logger LOG = LoggerFactory.getLogger(TestLogSegmentsZK.class); + static Logger LOG = Logger.get(TestLogSegmentsZK.class); private static MaxLogSegmentSequenceNo getMaxLogSegmentSequenceNo(ZooKeeperClient zkc, URI uri, String streamName, DistributedLogConfiguration conf) throws Exception { diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java index 8c89792445d..c87322a0589 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java @@ -17,7 +17,6 @@ */ package org.apache.distributedlog; - import static org.apache.distributedlog.NonBlockingReadsTestUtil.DEFAULT_SEGMENT_SIZE; import static org.apache.distributedlog.NonBlockingReadsTestUtil.readNonBlocking; import static org.apache.distributedlog.NonBlockingReadsTestUtil.writeRecordsForNonBlockingReads; @@ -28,6 +27,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.api.LogReader; import org.apache.distributedlog.exceptions.IdleReaderException; @@ -35,15 +35,13 @@ import org.junit.Assert; import org.junit.Ignore; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * {@link https://issues.apache.org/jira/browse/DL-12}. */ +@CustomLog @Ignore public class TestNonBlockingReads extends TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestNonBlockingReads.class); static { conf.setOutputBufferSize(0); @@ -114,7 +112,6 @@ public void run() { } }, 100, TimeUnit.MILLISECONDS); - readNonBlocking(dlm, false); assertFalse(currentThread.isInterrupted()); } finally { @@ -205,7 +202,7 @@ public void run() { try { readNonBlocking(dlm, false, 3, false); } catch (IdleReaderException exc) { - LOG.info("Exception encountered", exc); + log.info().exception(exc).log("Exception encountered"); exceptionEncountered = true; } assertFalse(exceptionEncountered); @@ -252,7 +249,6 @@ private long createStreamWithInconsistentMetadata(String name) throws Exception txid += 100; - for (long i = 0; i < 3; i++) { BKAsyncLogWriter out = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned(); for (long j = 1; j <= segmentSize; j++) { diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java index a90301d53e3..a6a7f5165c0 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java @@ -24,6 +24,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import lombok.CustomLog; import org.apache.distributedlog.api.AsyncLogWriter; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.api.LogReader; @@ -35,6 +36,7 @@ /** * Test Cases for NonBlockingReadsMultiReader. */ +@CustomLog public class TestNonBlockingReadsMultiReader extends TestDistributedLogBase { static class ReaderThread extends Thread { @@ -58,7 +60,8 @@ public void run() { if (r != null) { readCount.incrementAndGet(); if (readCount.get() % 1000 == 0) { - LOG.info("{} reading {}", getName(), r.getTransactionId()); + log.info().attr("reader", getName()).attr("transactionId", r.getTransactionId()) + .log("reading"); } } } catch (DLInterruptedException die) { @@ -70,14 +73,15 @@ public void run() { } void stopReading() { - LOG.info("Stopping reader."); + log.info("Stopping reader."); running = false; interrupt(); try { join(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOG.error("Interrupted on waiting reader thread {} exiting : ", getName(), e); + log.error().attr("reader", getName()).exception(e) + .log("Interrupted on waiting reader thread exiting"); } } @@ -126,10 +130,10 @@ public void run() { dlsn = Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(curTxId))); writeCount.incrementAndGet(); if (curTxId % 1000 == 0) { - LOG.info("writer write {}", curTxId); + log.info().attr("txid", curTxId).log("writer write"); } } - LOG.info("Completed writing record at {}", dlsn); + log.info().attr("dlsn", dlsn).log("Completed writing record"); Utils.close(writer); } catch (DLInterruptedException die) { Thread.currentThread().interrupt(); @@ -147,16 +151,17 @@ public void run() { TimeUnit.SECONDS.sleep(5); - LOG.info("Stopping writer"); + log.info("Stopping writer"); running.set(false); writerThread.join(); - LOG.info("Writer stopped after writing {} records, waiting for reader to complete", - writeCount.get()); + log.info().attr("writeCount", writeCount.get()) + .log("Writer stopped, waiting for reader to complete"); while (writeCount.get() > (readerThreads[0].getReadCount())) { - LOG.info("Write Count = {}, Read Count = {}", - new Object[] { writeCount.get(), readerThreads[0].getReadCount() }); + log.info().attr("writeCount", writeCount.get()) + .attr("readCount", readerThreads[0].getReadCount()) + .log("Progress"); TimeUnit.MILLISECONDS.sleep(100); } assertEquals(writeCount.get(), diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadUtils.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadUtils.java index 3fbcd700cf0..e6a7679d855 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadUtils.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadUtils.java @@ -27,23 +27,20 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; +import lombok.CustomLog; import org.apache.distributedlog.api.AsyncLogWriter; import org.apache.distributedlog.logsegment.LogSegmentFilter; import org.apache.distributedlog.util.Utils; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Test {@link ReadUtils}. */ +@CustomLog public class TestReadUtils extends TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestReadUtils.class); - @Rule public TestName runtime = new TestName(); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java index de880ee3d02..f62836bc9b8 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java @@ -29,22 +29,18 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.distributedlog.api.AsyncLogReader; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * A Reader wraps reading next logic for testing. */ +@CustomLog public class TestReader implements FutureEventListener { - static final Logger LOG = LoggerFactory.getLogger(TestReader.class); - final String readerName; final DistributedLogManager dlm; AsyncLogReader reader; @@ -124,7 +120,7 @@ public void run() { ((BKAsyncLogReader) reader).simulateErrors(); } nextDLSN = dlsn; - LOG.info("Positioned reader {} at {}", readerName, dlsn); + log.info().attr("readerName", readerName).attr("dlsn", dlsn).log("Positioned reader"); if (null != TestReader.this.reader) { Utils.close(TestReader.this.reader); } @@ -133,8 +129,8 @@ public void run() { readyLatch.countDown(); } catch (IOException exc) { int nextMs = nextDelayMs(); - LOG.info("Encountered exception {} on opening reader {} at {}, retrying in {} ms", - exc, readerName, dlsn, nextMs); + log.info().attr("readerName", readerName).attr("dlsn", dlsn).attr("nextMs", nextMs) + .exception(exc).log("Encountered exception on opening reader, retrying"); positionReader(dlsn); } } @@ -151,14 +147,14 @@ private void readNext() { public void onSuccess(LogRecordWithDLSN value) { try { assertTrue(value.getDlsn().compareTo(nextDLSN) >= 0); - LOG.info("Received record {} from log {} for reader {}", - value.getDlsn(), dlm.getStreamName(), readerName); + log.info().attr("dlsn", value.getDlsn()).attr("stream", dlm.getStreamName()) + .attr("readerName", readerName).log("Received record"); assertFalse(value.isControl()); assertEquals(0, value.getDlsn().getSlotId()); DLMTestUtil.verifyLargeLogRecord(value); } catch (Exception exc) { - LOG.error("Exception encountered when verifying received log record {} for reader {} :", - value.getDlsn(), readerName, exc); + log.error().attr("dlsn", value.getDlsn()).attr("readerName", readerName).exception(exc) + .log("Exception encountered when verifying received log record"); errorsFound.set(true); completionLatch.countDown(); return; @@ -166,12 +162,12 @@ public void onSuccess(LogRecordWithDLSN value) { readCount.incrementAndGet(); countLatch.countDown(); if (countLatch.getCount() <= 0) { - LOG.info("Reader {} is completed", readerName); + log.info().attr("readerName", readerName).log("Reader is completed"); closeReader(); completionLatch.countDown(); } else { - LOG.info("Reader {} : read count becomes {}, latch = {}", - readerName, readCount.get(), countLatch.getCount()); + log.info().attr("readerName", readerName).attr("readCount", readCount.get()) + .attr("latch", countLatch.getCount()).log("Reader read count"); nextDLSN = value.getDlsn().getNextDLSN(); readNext(); } @@ -179,7 +175,8 @@ public void onSuccess(LogRecordWithDLSN value) { @Override public void onFailure(Throwable cause) { - LOG.error("{} encountered exception on reading next record : ", readerName, cause); + log.error().attr("readerName", readerName).exception(cause) + .log("Reader encountered exception on reading next record"); closeReader(); nextDelayMs(); positionReader(nextDLSN); @@ -188,7 +185,7 @@ public void onFailure(Throwable cause) { private void closeReader() { if (null != reader) { reader.asyncClose().whenComplete((value, cause) -> { - LOG.warn("Exception on closing reader {} : ", readerName, cause); + log.warn().attr("readerName", readerName).exception(cause).log("Exception on closing reader"); }); } } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java index 1a8dc7d8a48..d892ddfb07c 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; +import lombok.CustomLog; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.common.concurrent.FutureEventListener; @@ -38,14 +39,12 @@ import org.apache.distributedlog.util.FailpointUtils; import org.apache.distributedlog.util.Utils; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Test Cases for RollLogSegments. */ +@CustomLog public class TestRollLogSegments extends TestDistributedLogBase { - private static final Logger logger = LoggerFactory.getLogger(TestRollLogSegments.class); private static void ensureOnlyOneInprogressLogSegments(List segments) throws Exception { int numInprogress = 0; @@ -85,7 +84,7 @@ public void testDisableRollingLogSegments() throws Exception { @Override public void onSuccess(DLSN value) { - logger.info("Completed entry {} : {}.", entryId, value); + log.info().attr("entryId", entryId).attr("dlsn", value).log("Completed entry"); latch.countDown(); } @@ -133,7 +132,7 @@ public void testLastDLSNInRollingLogSegments() throws Exception { @Override public void onSuccess(DLSN value) { - logger.info("Completed entry {} : {}.", entryId, value); + log.info().attr("entryId", entryId).attr("dlsn", value).log("Completed entry"); synchronized (lastDLSNs) { DLSN lastDLSN = lastDLSNs.get(value.getLogSegmentSequenceNo()); if (null == lastDLSN || lastDLSN.compareTo(value) < 0) { @@ -159,8 +158,8 @@ public void onFailure(Throwable cause) { writer.closeAndComplete(); List segments = dlm.getLogSegments(); - logger.info("lastDLSNs after writes {} {}", lastDLSNs.size(), lastDLSNs); - logger.info("segments after writes {} {}", segments.size(), segments); + log.info().attr("size", lastDLSNs.size()).attr("lastDLSNs", lastDLSNs).log("lastDLSNs after writes"); + log.info().attr("size", segments.size()).attr("segments", segments).log("segments after writes"); assertTrue(segments.size() >= 2); assertTrue(lastDLSNs.size() >= 2); assertEquals(lastDLSNs.size(), segments.size()); @@ -170,8 +169,8 @@ public void onFailure(Throwable cause) { assertNotNull(dlsnInMetadata); assertNotNull(dlsnSeen); if (dlsnInMetadata.compareTo(dlsnSeen) != 0) { - logger.error("Last dlsn recorded in log segment {} is different from the one already seen {}.", - dlsnInMetadata, dlsnSeen); + log.error().attr("dlsnInMetadata", dlsnInMetadata).attr("dlsnSeen", dlsnSeen) + .log("Last dlsn recorded in log segment differs from the one already seen"); } assertEquals(0, dlsnInMetadata.compareTo(dlsnSeen)); } @@ -208,12 +207,12 @@ public void testUnableToRollLogSegments() throws Exception { writer.write(DLMTestUtil.getLogRecordInstance(++txId)).whenComplete(new FutureEventListener() { @Override public void onSuccess(DLSN value) { - logger.info("Completed entry : {}.", value); + log.info().attr("entry", value).log("Completed entry"); latch.countDown(); } @Override public void onFailure(Throwable cause) { - logger.error("Failed to write entries : ", cause); + log.error().exception(cause).log("Failed to write entries"); } }); } @@ -223,7 +222,7 @@ public void onFailure(Throwable cause) { writer.close(); List segments = dlm.getLogSegments(); - logger.info("LogSegments: {}", segments); + log.info().attr("segments", segments).log("LogSegments"); assertEquals(1, segments.size()); @@ -250,7 +249,7 @@ record = reader.readNext(false); @Test(timeout = 60000) public void testRollingLogSegments() throws Exception { - logger.info("start testRollingLogSegments"); + log.info("start testRollingLogSegments"); String name = "distrlog-rolling-logsegments-hightraffic"; DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(conf); @@ -274,12 +273,12 @@ public void testRollingLogSegments() throws Exception { .whenComplete(new FutureEventListener() { @Override public void onSuccess(DLSN value) { - logger.info("Completed entry {} : {}.", entryId, value); + log.info().attr("entryId", entryId).attr("dlsn", value).log("Completed entry"); latch.countDown(); } @Override public void onFailure(Throwable cause) { - logger.error("Failed to write entries", cause); + log.error().exception(cause).log("Failed to write entries"); } }); if (i == 1) { @@ -289,11 +288,11 @@ public void onFailure(Throwable cause) { } latch.await(); - logger.info("Took {} ms to completed all requests.", System.currentTimeMillis() - startTime); + log.info().attr("elapsedMs", System.currentTimeMillis() - startTime).log("Took ms to complete all requests"); List segments = dlm.getLogSegments(); - logger.info("LogSegments : {}", segments); - logger.info("LogSegments size: {}", segments.size()); + log.info().attr("segments", segments).log("LogSegments"); + log.info().attr("size", segments.size()).log("LogSegments size"); assertTrue(segments.size() >= 2); ensureOnlyOneInprogressLogSegments(segments); @@ -304,12 +303,12 @@ public void onFailure(Throwable cause) { // there would be (numLogSegments/2) segments based on current rolling policy for (int i = 1; i <= numLogSegments; i++) { DLSN newDLSN = Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(numLogSegments + i))); - logger.info("Completed entry {} : {}", numLogSegments + i, newDLSN); + log.info().attr("entryId", numLogSegments + i).attr("dlsn", newDLSN).log("Completed entry"); } segments = dlm.getLogSegments(); - logger.info("LogSegments : {}", segments); - logger.info("LogSegments size: {}", segments.size()); + log.info().attr("segments", segments).log("LogSegments"); + log.info().attr("size", segments.size()).log("LogSegments size"); assertEquals(numSegmentsAfterAsyncWrites + numLogSegments / 2, segments.size()); ensureOnlyOneInprogressLogSegments(segments); @@ -327,11 +326,11 @@ private void checkAndWaitWriterReaderPosition(BKLogSegmentWriter writer, long ex assertEquals(expectedWriterPosition, getLedgerHandle(writer).getLastAddConfirmed()); assertEquals(expectedLac, inspector.readLastConfirmed()); EntryPosition readPosition = reader.getReadAheadReader().getNextEntryPosition(); - logger.info("ReadAhead moved read position {} : ", readPosition); + log.info().attr("readPosition", readPosition).log("ReadAhead moved read position"); while (readPosition.getEntryId() < expectedReaderPosition) { Thread.sleep(1000); readPosition = reader.getReadAheadReader().getNextEntryPosition(); - logger.info("ReadAhead moved read position {} : ", readPosition); + log.info().attr("readPosition", readPosition).log("ReadAhead moved read position"); } assertEquals(expectedReaderPosition, readPosition.getEntryId()); } @@ -404,7 +403,7 @@ public void testCaughtUpReaderOnLogSegmentRolling() throws Exception { } Thread.sleep(1000); } - logger.info("Waiting for long poll getting interrupted with metadata changed"); + log.info("Waiting for long poll getting interrupted with metadata changed"); // simulate a recovery without closing ledger causing recording wrong last dlsn BKLogWriteHandler writeHandler = writer.getCachedWriteHandler(); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestSequenceID.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestSequenceID.java index 377629e0ebb..dbd3796259e 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestSequenceID.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestSequenceID.java @@ -23,22 +23,19 @@ import java.util.List; import java.util.concurrent.LinkedBlockingQueue; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion; import org.apache.distributedlog.api.AsyncLogReader; import org.apache.distributedlog.util.Utils; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Test Cases related to sequence ids. */ +@CustomLog public class TestSequenceID extends TestDistributedLogBase { - private static final Logger logger = LoggerFactory.getLogger(TestSequenceID.class); - @Test(timeout = 60000) public void testCompleteV4LogSegmentAsV4() throws Exception { completeSingleInprogressSegment(LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value, @@ -143,7 +140,7 @@ public void onSuccess(LogRecordWithDLSN record) { @Override public void onFailure(Throwable cause) { - logger.error("Encountered exception on reading next : ", cause); + log.error().exception(cause).log("Encountered exception on reading next"); } }); } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTruncate.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTruncate.java index e32b5f2f7a5..7f5d01e2586 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTruncate.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTruncate.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.distributedlog.LogSegmentMetadata.TruncationStatus; @@ -36,14 +37,12 @@ import org.apache.distributedlog.api.LogReader; import org.apache.distributedlog.util.Utils; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Test Cases for truncation. */ +@CustomLog public class TestTruncate extends TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestTruncate.class); protected static DistributedLogConfiguration conf = new DistributedLogConfiguration() @@ -78,7 +77,7 @@ public void testPurgeLogs() throws Exception { DistributedLogManager distributedLogManager = createNewDLM(conf, name); List segments = distributedLogManager.getLogSegments(); - LOG.info("Segments before modifying completion time : {}", segments); + log.info().attr("segments", segments).log("Segments before modifying completion time"); ZooKeeperClient zkc = TestZooKeeperClientBuilder.newBuilder(conf) .uri(uri) @@ -93,7 +92,7 @@ public void testPurgeLogs() throws Exception { zkc.close(); segments = distributedLogManager.getLogSegments(); - LOG.info("Segments after modifying completion time : {}", segments); + log.info().attr("segments", segments).log("Segments after modifying completion time"); DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(conf); @@ -207,7 +206,7 @@ public void testOnlyPurgeSegmentsBeforeNoneFullyTruncatedSegment() throws Except DistributedLogManager dlm = createNewDLM(confLocal, name); List segments = dlm.getLogSegments(); - LOG.info("Segments before modifying segment status : {}", segments); + log.info().attr("segments", segments).log("Segments before modifying segment status"); ZooKeeperClient zkc = TestZooKeeperClientBuilder.newBuilder(conf) .uri(uri) @@ -221,7 +220,7 @@ public void testOnlyPurgeSegmentsBeforeNoneFullyTruncatedSegment() throws Except dlm.purgeLogsOlderThan(999999); List newSegments = dlm.getLogSegments(); - LOG.info("Segments after purge segments older than 999999 : {}", newSegments); + log.info().attr("newSegments", newSegments).log("Segments after purge segments older than 999999"); assertArrayEquals(segmentsAfterTruncated.toArray(new LogSegmentMetadata[segmentsAfterTruncated.size()]), newSegments.toArray(new LogSegmentMetadata[newSegments.size()])); @@ -247,7 +246,7 @@ public void testOnlyPurgeSegmentsBeforeNoneFullyTruncatedSegment() throws Except // to make sure the truncation task is executed DLSN lastDLSN = Utils.ioResult(newDLM.getLastDLSNAsync()); - LOG.info("Get last dlsn of stream {} : {}", name, lastDLSN); + log.info().attr("stream", name).attr("lastDLSN", lastDLSN).log("Get last dlsn of stream"); assertEquals(5, newDLM.getLogSegments().size()); @@ -272,7 +271,7 @@ public void testPartiallyTruncateTruncatedSegments() throws Exception { DistributedLogManager dlm = createNewDLM(confLocal, name); List segments = dlm.getLogSegments(); - LOG.info("Segments before modifying segment status : {}", segments); + log.info().attr("segments", segments).log("Segments before modifying segment status"); ZooKeeperClient zkc = TestZooKeeperClientBuilder.newBuilder(conf) .uri(uri) @@ -283,7 +282,7 @@ public void testPartiallyTruncateTruncatedSegments() throws Exception { } List newSegments = dlm.getLogSegments(); - LOG.info("Segments after changing truncation status : {}", newSegments); + log.info().attr("newSegments", newSegments).log("Segments after changing truncation status"); dlm.close(); @@ -306,7 +305,7 @@ private Pair populateData( boolean createInprogressLogSegment) throws Exception { long txid = 1; for (long i = 1; i <= numLogSegments; i++) { - LOG.info("Writing Log Segment {}.", i); + log.info().attr("segment", i).log("Writing Log Segment"); DistributedLogManager dlm = createNewDLM(confLocal, name); AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned(); for (int j = 1; j <= numEntriesPerLogSegment; j++) { diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java index 2eb6a61683b..e498160e1e3 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTxnId.java @@ -17,16 +17,15 @@ */ package org.apache.distributedlog; +import lombok.CustomLog; import org.junit.Assert; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Test Cases for RollLogSegments. */ +@CustomLog public class TestTxnId extends TestDistributedLogBase { - private static final Logger logger = LoggerFactory.getLogger(TestRollLogSegments.class); @Test public void testRecoveryAfterBookieCrash() throws Exception { @@ -70,7 +69,7 @@ public void testRecoveryAfterBookieCrash() throws Exception { BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned()) { long firstTxid = dlm.getLastTxId() + 1; for (int i = 0; i < 20; i++) { - logger.info("Writing entry {}", i); + log.info().attr("entry", i).log("Writing entry"); writer.write(DLMTestUtil.getLogRecordInstance(firstTxid + i, 100000)).join(); Thread.sleep(100); } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java index 898bc86fe22..19f5557a58a 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java @@ -17,24 +17,22 @@ */ package org.apache.distributedlog; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; +import lombok.CustomLog; import org.apache.bookkeeper.feature.Feature; import org.apache.bookkeeper.feature.SettableFeature; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.distributedlog.exceptions.OverCapacityException; import org.apache.distributedlog.util.SimplePermitLimiter; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Test Cases for {@link org.apache.distributedlog.util.SimplePermitLimiter}. */ +@CustomLog public class TestWriteLimiter { - static final Logger LOG = LoggerFactory.getLogger(TestWriteLimiter.class); SimplePermitLimiter createPermitLimiter(boolean darkmode, int permits) { return createPermitLimiter(darkmode, permits, new SettableFeature("", 0)); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java index a9ff1d1cbea..26d617aa296 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java @@ -30,6 +30,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; import org.apache.distributedlog.ZooKeeperClient.Credentials; @@ -48,16 +49,12 @@ import org.junit.Before; import org.junit.Ignore; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Test Cases for {@link org.apache.distributedlog.ZooKeeperClient}. */ +@CustomLog public class TestZooKeeperClient extends ZooKeeperClusterTestCase { - static final Logger LOG = LoggerFactory.getLogger(TestZooKeeperClient.class); private static final int sessionTimeoutMs = 2000; @@ -118,7 +115,7 @@ public void testAclCreatePerms() throws Exception { zkcNoAuth.get().create("/test/key2/key1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); fail("create should fail on acl protected key"); } catch (KeeperException.NoAuthException ex) { - LOG.info("caught exception writing to protected key", ex); + log.info().exception(ex).log("caught exception writing to protected key"); } rmAll(zkcAuth, "/test"); @@ -134,7 +131,7 @@ public void testAclNullIdDisablesAuth() throws Exception { DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT); fail("create should fail because we're not authenticated"); } catch (KeeperException.InvalidACLException ex) { - LOG.info("caught exception writing to protected key", ex); + log.info().exception(ex).log("caught exception writing to protected key"); } rmAll(zkcAuth, "/test"); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.java index 3e52d979f13..5bcaea9f457 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.java @@ -22,22 +22,18 @@ import com.google.common.base.Stopwatch; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooKeeper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Utilities of {@link org.apache.distributedlog.ZooKeeperClient}. */ +@CustomLog public class ZooKeeperClientUtils { - private static final Logger logger = LoggerFactory.getLogger(ZooKeeperClientUtils.class); - /** * Expire given zookeeper client's session. * @@ -57,7 +53,7 @@ public static void expireSession(ZooKeeperClient zkc, String zkServers, int time oldZk.exists("/", new Watcher() { @Override public void process(WatchedEvent event) { - logger.debug("Receive event : {}", event); + log.debug().attr("event", event).log("Receive event"); if (event.getType() == Event.EventType.None && event.getState() == Event.KeeperState.Expired) { expireLatch.countDown(); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/ZooKeeperClusterTestCase.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/ZooKeeperClusterTestCase.java index a3d59ef210e..538dadba441 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/ZooKeeperClusterTestCase.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/ZooKeeperClusterTestCase.java @@ -18,7 +18,7 @@ package org.apache.distributedlog; import java.io.File; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim; import org.apache.bookkeeper.util.IOUtils; import org.apache.commons.io.FileUtils; @@ -30,7 +30,7 @@ /** * ZooKeeperClusterTestCase. */ -@Slf4j +@CustomLog public class ZooKeeperClusterTestCase { static { @@ -52,12 +52,12 @@ public static void setupZooKeeper() throws Exception { zkPort = serverAndPort.getRight(); zkServers = "127.0.0.1:" + zkPort; - log.info("--- Setup zookeeper at {} ---", zkServers); + log.info().attr("zkServers", zkServers).log("--- Setup zookeeper ---"); } @AfterClass public static void shutdownZooKeeper() throws Exception { - log.info("--- Shutdown zookeeper at {} ---", zkServers); + log.info().attr("zkServers", zkServers).log("--- Shutdown zookeeper ---"); zks.stop(); if (null != zkDir) { FileUtils.forceDeleteOnExit(zkDir); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java index 940a18dcda6..18cf5030b65 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java @@ -22,6 +22,7 @@ import java.net.URI; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import lombok.CustomLog; import org.apache.distributedlog.DistributedLogConfiguration; import org.apache.distributedlog.TestZooKeeperClientBuilder; import org.apache.distributedlog.ZooKeeperClient; @@ -34,17 +35,13 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * TestZKAccessControlManager. */ +@CustomLog public class TestZKAccessControlManager extends ZooKeeperClusterTestCase { - private static final Logger logger = LoggerFactory.getLogger(TestZKAccessControlManager.class); - private DistributedLogConfiguration conf; private ZooKeeperClient zkc; private ScheduledExecutorService executorService; @@ -96,9 +93,9 @@ public void testZKAccessControlManager() throws Exception { String zkRootPath = "/test-zk-access-control-manager"; String stream1 = "test-acm-1"; String stream2 = "test-acm-2"; - logger.info("Creating ACL Manager for {}", zkRootPath); + log.info().attr("zkRootPath", zkRootPath).log("Creating ACL Manager"); ZKAccessControlManager zkcm = new ZKAccessControlManager(conf, zkc, zkRootPath, executorService); - logger.info("Created ACL Manager for {}", zkRootPath); + log.info().attr("zkRootPath", zkRootPath).log("Created ACL Manager"); try { verifyStreamPermissions(zkcm, stream1, true, true, true, true, true); @@ -108,7 +105,7 @@ public void testZKAccessControlManager() throws Exception { ace1.setDenyDelete(true); ZKAccessControl accessControl1 = new ZKAccessControl(ace1, zkPath1); setACL(accessControl1); - logger.info("Create ACL for stream {} : {}", stream1, accessControl1); + log.info().attr("stream", stream1).attr("accessControl", accessControl1).log("Create ACL for stream"); while (zkcm.allowDelete(stream1)) { Thread.sleep(100); } @@ -119,7 +116,7 @@ public void testZKAccessControlManager() throws Exception { ace1.setDenyWrite(true); accessControl1 = new ZKAccessControl(ace1, zkPath1); setACL(accessControl1); - logger.info("Update ACL for stream {} : {}", stream1, accessControl1); + log.info().attr("stream", stream1).attr("accessControl", accessControl1).log("Update ACL for stream"); // create stream2 (denyTruncate = true) String zkPath2 = zkRootPath + "/" + stream2; @@ -127,7 +124,7 @@ public void testZKAccessControlManager() throws Exception { ace2.setDenyTruncate(true); ZKAccessControl accessControl2 = new ZKAccessControl(ace2, zkPath2); setACL(accessControl2); - logger.info("Create ACL for stream {} : {}", stream2, accessControl2); + log.info().attr("stream", stream2).attr("accessControl", accessControl2).log("Create ACL for stream"); while (zkcm.allowWrite(stream1)) { Thread.sleep(100); } @@ -140,7 +137,7 @@ public void testZKAccessControlManager() throws Exception { // delete stream2 Utils.ioResult(ZKAccessControl.delete(zkc, zkPath2)); - logger.info("Delete ACL for stream {}", stream2); + log.info().attr("stream", stream2).log("Delete ACL for stream"); while (!zkcm.allowTruncate(stream2)) { Thread.sleep(100); } @@ -156,14 +153,17 @@ public void testZKAccessControlManager() throws Exception { ace1.setDenyRelease(true); accessControl1 = new ZKAccessControl(ace1, zkPath1); setACL(accessControl1); - logger.info("Update ACL for stream {} : {}", stream1, accessControl1); + log.info().attr("stream", stream1).attr("accessControl", accessControl1).log("Update ACL for stream"); // create stream2 (denyTruncate = true) ace2 = new AccessControlEntry(); ace2.setDenyAcquire(true); accessControl2 = new ZKAccessControl(ace2, zkPath2); setACL(accessControl2); - logger.info("Created ACL for stream {} again : {}", stream2, accessControl2); + log.info() + .attr("stream", stream2) + .attr("accessControl", accessControl2) + .log("Created ACL for stream again"); while (zkcm.allowRelease(stream1)) { Thread.sleep(100); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java index bed16ad5c0f..ba7eec4dd28 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java @@ -27,6 +27,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.distributedlog.DLMTestUtil; import org.apache.distributedlog.DLSN; @@ -46,20 +47,13 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - - - /** * TestDLCK. */ +@CustomLog public class TestDLCK extends TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestDLCK.class); - protected static DistributedLogConfiguration conf = new DistributedLogConfiguration().setLockTimeout(10) .setEnableLedgerAllocatorPool(true).setLedgerAllocatorPoolName("test"); @@ -139,7 +133,7 @@ public void testCheckAndRepairDLNamespace() throws Exception { false); Map segments = getLogSegments(dlm); - LOG.info("segments after drynrun {}", segments); + log.info().attr("segments", segments).log("segments after dryrun"); verifyLogSegment(segments, new DLSN(1L, 18L, 0L), 1L, 10, 10L); verifyLogSegment(segments, new DLSN(2L, 16L, 0L), 2L, 9, 19L); verifyLogSegment(segments, new DLSN(3L, 18L, 0L), 3L, 10, 30L); @@ -155,7 +149,7 @@ public void testCheckAndRepairDLNamespace() throws Exception { false); segments = getLogSegments(dlm); - LOG.info("segments after repair {}", segments); + log.info().attr("segments", segments).log("segments after repair"); verifyLogSegment(segments, new DLSN(1L, 18L, 0L), 1L, 10, 10L); verifyLogSegment(segments, new DLSN(2L, 18L, 0L), 2L, 10, 20L); verifyLogSegment(segments, new DLSN(3L, 18L, 0L), 3L, 10, 30L); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java index e4c9a77ce0f..8c58722d09a 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java @@ -23,6 +23,7 @@ import java.net.URI; import java.util.concurrent.CompletableFuture; +import lombok.CustomLog; import org.apache.bookkeeper.common.testing.annotations.FlakyTest; import org.apache.distributedlog.DLMTestUtil; import org.apache.distributedlog.DLSN; @@ -44,16 +45,13 @@ import org.apache.zookeeper.ZooDefs; import org.junit.After; import org.junit.Before; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * TestDistributedLogAdmin. */ +@CustomLog public class TestDistributedLogAdmin extends TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestDistributedLogAdmin.class); - private ZooKeeperClient zooKeeperClient; @Before @@ -118,12 +116,12 @@ public void testChangeSequenceNumber() throws Exception { lastDLSN = record.getDlsn(); } - LOG.info("Injecting bad log segment '3'"); + log.info("Injecting bad log segment '3'"); dlm = namespace.openLog(streamName); DLMTestUtil.injectLogSegmentWithGivenLogSegmentSeqNo(dlm, confLocal, 3L, 5 * 10 + 1, true, 10, false); - LOG.info("Injected bad log segment '3'"); + log.info("Injected bad log segment '3'"); // there isn't records should be read CompletableFuture readFuture = reader.readNext(); @@ -136,7 +134,7 @@ public void testChangeSequenceNumber() throws Exception { // expected } - LOG.info("Dryrun fix inprogress segment that has lower sequence number"); + log.info("Dryrun fix inprogress segment that has lower sequence number"); // Dryrun DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(namespace, @@ -151,7 +149,7 @@ public void testChangeSequenceNumber() throws Exception { // expected } - LOG.info("Actual run fix inprogress segment that has lower sequence number"); + log.info("Actual run fix inprogress segment that has lower sequence number"); // Actual run DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(namespace, diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java index 422a765f80c..db4b3203b2c 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java @@ -29,6 +29,7 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerEntry; @@ -59,17 +60,14 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * TestLedgerAllocator. */ +@CustomLog @TestInstance(TestInstance.Lifecycle.PER_CLASS) public class TestLedgerAllocator extends TestDistributedLogBase { - private static final Logger logger = LoggerFactory.getLogger(TestLedgerAllocator.class); - private static final String ledgersPath = "/ledgers"; private static final OpListener NULL_LISTENER = new OpListener() { @Override @@ -138,7 +136,7 @@ public void testAllocation() throws Exception { allocator.allocate(); ZKTransaction txn = newTxn(); LedgerHandle lh = Utils.ioResult(allocator.tryObtain(txn, NULL_LISTENER)); - logger.info("Try obtaining ledger handle {}", lh.getId()); + log.info().attr("ledgerId", lh.getId()).log("Try obtaining ledger handle"); byte[] data = zkc.get().getData(allocationPath, false, null); assertEquals((Long) lh.getId(), Long.valueOf(new String(data, UTF_8))); txn.addOp(DefaultZKOp.of(Op.setData("/unexistedpath", "data".getBytes(UTF_8), -1), null)); @@ -147,7 +145,7 @@ public void testAllocation() throws Exception { fail("Should fail the transaction when setting unexisted path"); } catch (ZKException ke) { // expected - logger.info("Should fail on executing transaction when setting unexisted path", ke); + log.info().exception(ke).log("Should fail on executing transaction when setting unexisted path"); } data = zkc.get().getData(allocationPath, false, null); assertEquals((Long) lh.getId(), Long.valueOf(new String(data, UTF_8))); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java index 2d4f6d47807..5cde64fe2e0 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java @@ -31,6 +31,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicInteger; +import lombok.CustomLog; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.distributedlog.BookKeeperClient; import org.apache.distributedlog.BookKeeperClientBuilder; @@ -47,19 +48,13 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - - /** * TestLedgerAllocatorPool. */ +@CustomLog public class TestLedgerAllocatorPool extends TestDistributedLogBase { - private static final Logger logger = LoggerFactory.getLogger(TestLedgerAllocatorPool.class); - private static final String ledgersPath = "/ledgers"; private static final OpListener NULL_LISTENER = new OpListener() { @Override @@ -170,7 +165,7 @@ public void testRescueAllocators() throws Exception { SimpleLedgerAllocator sla = pool.getLedgerAllocator(lh); String slaPath = sla.allocatePath; - logger.info("Allocated ledger {} from path {}", lh.getId(), slaPath); + log.info().attr("ledgerId", lh.getId()).attr("slaPath", slaPath).log("Allocated ledger from path"); pendingTxns.add(txn); allocatePaths.add(slaPath); @@ -199,7 +194,7 @@ public void testRescueAllocators() throws Exception { SimpleLedgerAllocator sla = pool.getLedgerAllocator(lh); String slaPath = sla.allocatePath; - logger.info("Allocated ledger {} from path {}", lh.getId(), slaPath); + log.info().attr("ledgerId", lh.getId()).attr("slaPath", slaPath).log("Allocated ledger from path"); allocatedPathSet.add(slaPath); Utils.ioResult(txn.execute()); @@ -289,7 +284,8 @@ public void run() { Utils.ioResult(txn.execute()); lh.close(); allocatedLedgers.putIfAbsent(lh.getId(), lh); - logger.info("[thread {}] allocate {}th ledger {}", tid, i, lh.getId()); + log.info().attr("thread", tid).attr("ledgerIndex", i).attr("ledgerId", lh.getId()) + .log("thread allocate ledger"); } } catch (Exception ioe) { numFailures.incrementAndGet(); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/PropertiesWriter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/PropertiesWriter.java index f49d0cc3794..a072b72d363 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/PropertiesWriter.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/PropertiesWriter.java @@ -20,14 +20,13 @@ import java.io.File; import java.io.FileOutputStream; import java.util.Properties; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import lombok.CustomLog; /** * Writer to write properties to files. */ +@CustomLog public class PropertiesWriter { - static final Logger LOG = LoggerFactory.getLogger(PropertiesWriter.class); final FileOutputStream outputStream; final File configFile; @@ -60,9 +59,7 @@ public void save() throws Exception { FileOutputStream outputStream = new FileOutputStream(configFile); properties.store(outputStream, null); configFile.setLastModified(configFile.lastModified() + 1000); - if (LOG.isDebugEnabled()) { - LOG.debug("save modified={}", configFile.lastModified()); - } + log.debug().attr("modified", configFile.lastModified()).log("save"); } public File getFile() { diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java index d8bac3cd49f..4bb7c129344 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java @@ -25,20 +25,18 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import lombok.CustomLog; import org.apache.distributedlog.DistributedLogConfiguration; import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration; import org.apache.distributedlog.common.config.ConcurrentConstConfiguration; import org.apache.distributedlog.common.config.PropertiesWriter; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * TestDynamicConfigurationFactory. */ +@CustomLog public class TestDynamicConfigurationFactory { - static final Logger LOG = LoggerFactory.getLogger(TestDynamicConfigurationFactory.class); private void waitForConfig(DynamicDistributedLogConfiguration conf, int value) throws Exception { while (!Objects.equal(conf.getRetentionPeriodHours(), value)) { diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java index cefee7d783d..5f86843e0d7 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java @@ -27,23 +27,17 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import lombok.CustomLog; import org.apache.distributedlog.DLMTestUtil; import org.apache.distributedlog.DistributedLogConstants; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - - - /** * TestZKLogSegmentFilters. */ +@CustomLog public class TestZKLogSegmentFilters { - static final Logger LOG = LoggerFactory.getLogger(TestZKLogSegmentFilters.class); - @Test(timeout = 60000) public void testWriteFilter() { Set expectedFilteredSegments = new HashSet(); @@ -71,7 +65,8 @@ public void testWriteFilter() { expectedFilteredSegments.add(DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX + "_1_2_3_4_5_6_7_8_9"); Collection filteredCollection = WRITE_HANDLE_FILTER.filter(segments); - LOG.info("Filter log segments {} to {}.", segments, filteredCollection); + log.info().attr("segments", segments).attr("filteredCollection", filteredCollection) + .log("Filter log segments"); assertEquals(expectedFilteredSegments.size(), filteredCollection.size()); Set filteredSegments = Sets.newHashSet(filteredCollection); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java index f466a781d30..d4ee2643eb1 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java @@ -33,6 +33,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.versioning.LongVersion; @@ -61,16 +62,13 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Test ZK based log segment metadata store. */ +@CustomLog public class TestZKLogSegmentMetadataStore extends TestDistributedLogBase { - private static final Logger logger = LoggerFactory.getLogger(TestZKLogSegmentMetadataStore.class); - private static final int zkSessionTimeoutMs = 2000; private LogSegmentMetadata createLogSegment( @@ -371,7 +369,7 @@ public void testLogSegmentNamesListener() throws Exception { LogSegmentNamesListener listener = new LogSegmentNamesListener() { @Override public void onSegmentsUpdated(Versioned> segments) { - logger.info("Received segments : {}", segments); + log.info().attr("segments", segments).log("Received segments"); segmentLists.add(segments.getValue()); numNotifications.incrementAndGet(); } @@ -394,7 +392,7 @@ public void onLogStreamDeleted() { assertEquals("List of segments should be same", children, firstSegmentList); - logger.info("Create another {} segments.", numSegments); + log.info().attr("numSegments", numSegments).log("Create another segments"); // create another log segment, it should trigger segment list updated Transaction anotherCreateTxn = lsmStore.transaction(); @@ -405,7 +403,7 @@ public void onLogStreamDeleted() { Utils.ioResult(anotherCreateTxn.execute()); List newChildren = zkc.get().getChildren(rootPath, false); Collections.sort(newChildren); - logger.info("All log segments become {}", newChildren); + log.info().attr("newChildren", newChildren).log("All log segments become"); while (numNotifications.get() < 2) { TimeUnit.MILLISECONDS.sleep(10); } @@ -437,7 +435,7 @@ public void testLogSegmentNamesListenerOnDeletion() throws Exception { LogSegmentNamesListener listener = new LogSegmentNamesListener() { @Override public void onSegmentsUpdated(Versioned> segments) { - logger.info("Received segments : {}", segments); + log.info().attr("segments", segments).log("Received segments"); segmentLists.add(segments.getValue()); numNotifications.incrementAndGet(); } @@ -509,7 +507,7 @@ public void testLogSegmentNamesListenerOnSessionExpired() throws Exception { LogSegmentNamesListener listener = new LogSegmentNamesListener() { @Override public void onSegmentsUpdated(Versioned> segments) { - logger.info("Received segments : {}", segments); + log.info().attr("segments", segments).log("Received segments"); segmentLists.add(segments.getValue()); numNotifications.incrementAndGet(); } @@ -536,7 +534,7 @@ public void onLogStreamDeleted() { ZooKeeperClientUtils.expireSession(zkc, BKNamespaceDriver.getZKServersFromDLUri(uri), conf.getZKSessionTimeoutMilliseconds()); - logger.info("Create another {} segments.", numSegments); + log.info().attr("numSegments", numSegments).log("Create another segments"); // create another log segment, it should trigger segment list updated Transaction anotherCreateTxn = lsmStore.transaction(); @@ -547,7 +545,7 @@ public void onLogStreamDeleted() { Utils.ioResult(anotherCreateTxn.execute()); List newChildren = zkc.get().getChildren(rootPath, false); Collections.sort(newChildren); - logger.info("All log segments become {}", newChildren); + log.info().attr("newChildren", newChildren).log("All log segments become"); while (numNotifications.get() < 2) { TimeUnit.MILLISECONDS.sleep(10); } @@ -580,7 +578,7 @@ public void testLogSegmentNamesListenerOnDeletingLogStream() throws Exception { LogSegmentNamesListener listener = new LogSegmentNamesListener() { @Override public void onSegmentsUpdated(Versioned> segments) { - logger.info("Received segments : {}", segments); + log.info().attr("segments", segments).log("Received segments"); segmentLists.add(segments.getValue()); numNotifications.incrementAndGet(); } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java index d4a6764a02d..b7d0baa3958 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java @@ -48,6 +48,7 @@ import java.net.URI; import java.util.Collections; import java.util.List; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.stats.NullStatsLogger; @@ -86,16 +87,13 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Test {@link ZKLogStreamMetadataStore}. */ +@CustomLog public class TestZKLogStreamMetadataStore extends ZooKeeperClusterTestCase { - private static final Logger logger = LoggerFactory.getLogger(TestZKLogStreamMetadataStore.class); - private static final int sessionTimeoutMs = 30000; @Rule @@ -202,7 +200,7 @@ public void setup() throws Exception { ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); } catch (KeeperException.NodeExistsException nee) { - logger.debug("The namespace uri already exists."); + log.debug("The namespace uri already exists."); } scheduler = OrderedScheduler.newSchedulerBuilder() .name("test-scheduler") diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java index 43e9aca052f..8b25e7a27c5 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java @@ -31,6 +31,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.stats.NullStatsLogger; @@ -52,17 +53,13 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Distributed Lock Tests. */ +@CustomLog public class TestDistributedLock extends TestDistributedLogBase { - private static final Logger logger = LoggerFactory.getLogger(TestDistributedLock.class); - @Rule public TestName runtime = new TestName(); @@ -347,7 +344,6 @@ public void testCheckWriteLockFailureWhenLockIsAcquiredByOthers() throws Excepti assertEquals(lockId0_2, Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0)))); - SessionLockFactory lockFactory = createLockFactory(clientId, zkc); final ZKDistributedLock lock1 = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath, @@ -360,7 +356,7 @@ public void run() { Utils.ioResult(lock1.asyncAcquire()); lockLatch.countDown(); } catch (Exception e) { - logger.error("Failed on locking lock1 : ", e); + log.error().exception(e).log("Failed on locking lock1"); } } }, "lock-thread"); @@ -507,7 +503,7 @@ public void run() { Utils.ioResult(lock1.asyncAcquire()); lock1DoneLatch.countDown(); } catch (Exception e) { - logger.error("Error on acquiring lock1 : ", e); + log.error().exception(e).log("Error on acquiring lock1"); } } }, "lock1-thread"); @@ -526,9 +522,9 @@ public void run() { assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(1)))); - logger.info("Expiring session on lock0"); + log.info("Expiring session on lock0"); ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs); - logger.info("Session on lock0 is expired"); + log.info("Session on lock0 is expired"); lock1DoneLatch.await(); assertFalse(lock0.haveLock()); assertTrue(lock1.haveLock()); @@ -549,7 +545,7 @@ public void run() { oafe.getCurrentOwner()); } } else { - logger.info("Waiting lock0 to attempt acquisition after session expired"); + log.info("Waiting lock0 to attempt acquisition after session expired"); // session expire will trigger lock re-acquisition CompletableFuture asyncLockAcquireFuture; do { diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java index deba668389a..098dd2d00a8 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java @@ -38,6 +38,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import lombok.CustomLog; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.commons.lang3.tuple.Pair; @@ -60,19 +61,16 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Distributed Lock Tests. */ +@CustomLog public class TestZKSessionLock extends ZooKeeperClusterTestCase { @Rule public TestName testNames = new TestName(); - private static final Logger logger = LoggerFactory.getLogger(TestZKSessionLock.class); - private static final int sessionTimeoutMs = 2000; private ZooKeeperClient zkc; @@ -666,7 +664,7 @@ public void run() { lock1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS); lock1DoneLatch.countDown(); } catch (LockingException e) { - logger.error("Failed on locking lock1 : ", e); + log.error().exception(e).log("Failed on locking lock1"); } } }, "lock1-thread"); @@ -820,7 +818,7 @@ public void run() { } catch (OwnershipAcquireFailedException oafe) { lock1DoneLatch.countDown(); } catch (LockingException e) { - logger.error("Failed on locking lock1 : ", e); + log.error().exception(e).log("Failed on locking lock1"); } } }, "lock1-thread"); @@ -906,7 +904,7 @@ public void run() { lock1_1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS); lock1DoneLatch.countDown(); } catch (LockingException e) { - logger.error("Failed on locking lock1 : ", e); + log.error().exception(e).log("Failed on locking lock1"); } } }, "lock1-thread"); @@ -915,7 +913,7 @@ public void run() { // check lock1 is waiting for lock0 children = awaitWaiters(2, zkc, lockPath); - logger.info("Found {} lock waiters : {}", children.size(), children); + log.info().attr("numWaiters", children.size()).attr("children", children).log("Found lock waiters"); assertEquals(2, children.size()); assertEquals(State.CLAIMED, lock0.getLockState()); @@ -1031,7 +1029,7 @@ public void run() { lock1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS); lock1DoneLatch.countDown(); } catch (LockingException e) { - logger.error("Failed on locking lock1 : ", e); + log.error().exception(e).log("Failed on locking lock1"); } } }, "lock1-thread"); @@ -1079,7 +1077,7 @@ public void run() { lock0DoneLatch.countDown(); } } catch (LockingException le) { - logger.error("Failed on locking lock0_1 : ", le); + log.error().exception(le).log("Failed on locking lock0_1"); } } }, "lock0-thread"); diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java index a7b6f7fa987..b31cc32be3b 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java @@ -25,6 +25,7 @@ import java.net.URI; import java.util.HashMap; import java.util.Map; +import lombok.CustomLog; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.distributedlog.DLMTestUtil; import org.apache.distributedlog.DLSN; @@ -42,18 +43,13 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * Test update for {@link LogSegmentMetadataStore}s. */ +@CustomLog public class TestLogSegmentMetadataStoreUpdater extends ZooKeeperClusterTestCase { - static final Logger LOG = LoggerFactory.getLogger(TestLogSegmentMetadataStoreUpdater.class); - private ZooKeeperClient zkc; private OrderedScheduler scheduler; private LogSegmentMetadataStore metadataStore; @@ -98,14 +94,14 @@ public void testChangeSequenceNumber() throws Exception { LogSegmentMetadata segment = DLMTestUtil.completedLogSegment(ledgerPath, i, (i - 1) * 100, i * 100 - 1, 100, i, 100, 0); completedLogSegments.put(((long) i), segment); - LOG.info("Create completed segment {} : {}", segment.getZkPath(), segment); + log.info().attr("zkPath", segment.getZkPath()).attr("segment", segment).log("Create completed segment"); segment.write(zkc); } // Create a smaller inprogress log segment long inprogressSeqNo = 3; LogSegmentMetadata segment = DLMTestUtil.inprogressLogSegment(ledgerPath, inprogressSeqNo, 5 * 100, inprogressSeqNo); - LOG.info("Create inprogress segment {} : {}", segment.getZkPath(), segment); + log.info().attr("zkPath", segment.getZkPath()).attr("segment", segment).log("Create inprogress segment"); segment.write(zkc); Map segmentList = readLogSegments(ledgerPath); @@ -245,7 +241,7 @@ public void testChangeTruncationStatus() throws Exception { LogSegmentMetadata segment = DLMTestUtil.completedLogSegment(ledgerPath, i, (i - 1) * 100, i * 100 - 1, 100, i, 100, 0); completedLogSegments.put(((long) i), segment); - LOG.info("Create completed segment {} : {}", segment.getZkPath(), segment); + log.info().attr("zkPath", segment.getZkPath()).attr("segment", segment).log("Create completed segment"); segment.write(zkc); } diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java index ab0ad2f33f0..763d1a281cd 100644 --- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java +++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java @@ -22,6 +22,7 @@ import static org.junit.Assert.fail; import java.net.URI; +import lombok.CustomLog; import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsOnMetadataServerException; import org.apache.bookkeeper.common.util.ReflectionUtils; import org.apache.distributedlog.DLMTestUtil; @@ -47,17 +48,13 @@ import org.apache.zookeeper.KeeperException; import org.junit.BeforeClass; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Test Case for {@link DistributedLogTool}s. */ +@CustomLog public class TestDistributedLogTool extends TestDistributedLogBase { - static final Logger LOG = LoggerFactory.getLogger(TestDistributedLogTool.class); - private static final String defaultLedgerPath = LocalDLMEmulator.getBkLedgerPath(); private static final String defaultPath = "/test/namespace"; private static final String defaultHost = "127.0.0.1"; diff --git a/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLFileSystem.java b/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLFileSystem.java index 8b09db07d8f..0f747ec924d 100644 --- a/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLFileSystem.java +++ b/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLFileSystem.java @@ -30,7 +30,7 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.ExecutionException; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.commons.configuration2.ex.ConfigurationException; import org.apache.distributedlog.DLSN; import org.apache.distributedlog.DistributedLogConfiguration; @@ -57,7 +57,7 @@ /** * A FileSystem Implementation powered by replicated logs. */ -@Slf4j +@CustomLog public class DLFileSystem extends FileSystem { // @@ -99,13 +99,14 @@ public void initialize(URI name, Configuration conf) throws IOException { if (null != dlConfLocation) { try { this.dlConf.loadConf(new File(dlConfLocation).toURI().toURL()); - log.info("Loaded the distributedlog configuration from {}", dlConfLocation); + log.info().attr("dlConfLocation", dlConfLocation).log("Loaded the distributedlog configuration"); } catch (ConfigurationException e) { - log.error("Failed to load the distributedlog configuration from " + dlConfLocation, e); + log.error().attr("dlConfLocation", dlConfLocation).exception(e) + .log("Failed to load the distributedlog configuration"); throw new IOException("Failed to load distributedlog configuration from " + dlConfLocation); } } - log.info("Initializing the filesystem at {}", name); + log.info().attr("name", name).log("Initializing the filesystem"); // initialize the namespace this.namespace = NamespaceBuilder.newBuilder() .clientId("dlfs-client-" + InetAddress.getLocalHost().getHostName()) @@ -113,7 +114,7 @@ public void initialize(URI name, Configuration conf) throws IOException { .regionId(DistributedLogConstants.LOCAL_REGION_ID) .uri(name) .build(); - log.info("Initialized the filesystem at {}", name); + log.info().attr("name", name).log("Initialized the filesystem"); } @Override diff --git a/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLInputStream.java b/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLInputStream.java index c5a810fffb9..b80a64c0bb3 100644 --- a/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLInputStream.java +++ b/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLInputStream.java @@ -22,7 +22,7 @@ import java.io.IOException; import java.io.InputStream; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.distributedlog.LogRecordWithDLSN; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.api.LogReader; @@ -31,7 +31,7 @@ /** * The input stream for a distributedlog stream. */ -@Slf4j +@CustomLog class DLInputStream extends FSInputStream { private static final long REOPEN_READER_SKIP_BYTES = 4 * 1024 * 1024; // 4MB diff --git a/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLOutputStream.java b/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLOutputStream.java index 68c3d087385..7fb18406975 100644 --- a/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLOutputStream.java +++ b/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLOutputStream.java @@ -26,7 +26,7 @@ import java.io.OutputStream; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.distributedlog.DLSN; @@ -39,7 +39,7 @@ /** * DistributedLog Output Stream. */ -@Slf4j +@CustomLog class DLOutputStream extends OutputStream { private static final byte[] CONTROL_RECORD_CONTENT = "control".getBytes(UTF_8); @@ -127,7 +127,7 @@ public void flush() throws IOException { } catch (IOException ioe) { throw ioe; } catch (Exception e) { - log.error("Unexpected exception in DLOutputStream", e); + log.error().exception(e).log("Unexpected exception in DLOutputStream"); throw new UnexpectedException("unexpected exception in DLOutputStream#flush()", e); } } diff --git a/stream/distributedlog/io/dlfs/src/test/java/org/apache/distributedlog/fs/TestDLFileSystem.java b/stream/distributedlog/io/dlfs/src/test/java/org/apache/distributedlog/fs/TestDLFileSystem.java index 4a2fc9e321d..b940abbfe81 100644 --- a/stream/distributedlog/io/dlfs/src/test/java/org/apache/distributedlog/fs/TestDLFileSystem.java +++ b/stream/distributedlog/io/dlfs/src/test/java/org/apache/distributedlog/fs/TestDLFileSystem.java @@ -28,7 +28,7 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.InputStreamReader; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; @@ -40,7 +40,7 @@ /** * Integration test for {@link DLFileSystem}. */ -@Slf4j +@CustomLog public class TestDLFileSystem extends TestDLFSBase { @Rule diff --git a/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetReader.java b/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetReader.java index 1f147f57250..a5e0f375be1 100644 --- a/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetReader.java +++ b/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetReader.java @@ -24,7 +24,7 @@ import io.netty.buffer.ByteBuf; import io.netty.util.ReferenceCountUtil; import java.io.IOException; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.distributedlog.io.CompressionCodec; import org.apache.distributedlog.io.CompressionCodec.Type; import org.apache.distributedlog.io.CompressionUtils; @@ -32,7 +32,7 @@ /** * Record reader to read records from an enveloped entry buffer. */ -@Slf4j +@CustomLog class EnvelopedRecordSetReader implements LogRecordSet.Reader { private final long logSegmentSeqNo; diff --git a/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetWriter.java b/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetWriter.java index f1ba4c40da2..62c81d25f2a 100644 --- a/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetWriter.java +++ b/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetWriter.java @@ -35,7 +35,7 @@ import java.util.LinkedList; import java.util.List; import java.util.concurrent.CompletableFuture; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.distributedlog.exceptions.LogRecordTooLongException; import org.apache.distributedlog.exceptions.WriteException; import org.apache.distributedlog.io.CompressionCodec; @@ -45,7 +45,7 @@ /** * {@link ByteBuf} based log record set writer. */ -@Slf4j +@CustomLog class EnvelopedRecordSetWriter implements LogRecordSet.Writer { private final ByteBuf buffer; diff --git a/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/LogRecord.java b/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/LogRecord.java index 7101420675c..e46e96ab930 100644 --- a/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/LogRecord.java +++ b/stream/distributedlog/protocol/src/main/java/org/apache/distributedlog/LogRecord.java @@ -27,9 +27,8 @@ import java.io.InputStream; import java.nio.ByteBuffer; import javax.annotation.concurrent.NotThreadSafe; +import lombok.CustomLog; import org.apache.distributedlog.common.util.ByteBufUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Log record is the basic element in a log. @@ -107,11 +106,10 @@ * * @see LogRecordWithDLSN */ +@CustomLog @NotThreadSafe public class LogRecord { - private static final Logger LOG = LoggerFactory.getLogger(LogRecord.class); - // Allow 4K overhead for metadata within the max transmission size public static final int MAX_LOGRECORD_SIZE = 1024 * 1024 - 8 * 1024; //1MB - 8KB // Allow 4K overhead for transmission overhead @@ -524,14 +522,12 @@ public LogRecordWithDLSN readOp() throws IOException { // don't have to deal with reference count. boolean copyData = !isRecordSet(metadata) || !deserializeRecordSet; nextRecordInStream.readPayload(in, copyData); - if (LOG.isTraceEnabled()) { - if (nextRecordInStream.isControl()) { - LOG.trace("Reading {} Control DLSN {}", - recordStream.getName(), nextRecordInStream.getDlsn()); - } else { - LOG.trace("Reading {} Valid DLSN {}", - recordStream.getName(), nextRecordInStream.getDlsn()); - } + if (nextRecordInStream.isControl()) { + log.trace().attr("stream", recordStream.getName()) + .attr("dlsn", nextRecordInStream.getDlsn()).log("Reading Control DLSN"); + } else { + log.trace().attr("stream", recordStream.getName()) + .attr("dlsn", nextRecordInStream.getDlsn()).log("Reading Valid DLSN"); } int numRecords = 1; @@ -586,9 +582,8 @@ private boolean skipTo(Long txId, DLSN dlsn, boolean skipControl) throws IOExcep } if ((null != dlsn) && (recordStream.getCurrentPosition().compareTo(dlsn) >= 0)) { - if (LOG.isTraceEnabled()) { - LOG.trace("Found position {} beyond {}", recordStream.getCurrentPosition(), dlsn); - } + log.trace().attr("position", recordStream.getCurrentPosition()).attr("dlsn", dlsn) + .log("Found position beyond dlsn"); if (null == lastRecordSkipTo) { in.resetReaderIndex(); } @@ -597,9 +592,8 @@ private boolean skipTo(Long txId, DLSN dlsn, boolean skipControl) throws IOExcep } if ((null != txId) && (currTxId >= txId)) { if (!skipControl || !isControl(flags)) { - if (LOG.isTraceEnabled()) { - LOG.trace("Found position {} beyond {}", currTxId, txId); - } + log.trace().attr("currTxId", currTxId).attr("txId", txId) + .log("Found position beyond txId"); if (null == lastRecordSkipTo) { in.resetReaderIndex(); } @@ -627,21 +621,17 @@ private boolean skipTo(Long txId, DLSN dlsn, boolean skipControl) throws IOExcep if (length < 0) { // We should never really see this as we only write complete entries to // BK and BK client has logic to detect torn writes (through checksum) - LOG.info("Encountered Record with negative length at TxId: {}", currTxId); + log.info().attr("txId", currTxId).log("Encountered Record with negative length"); break; } // skip single record in.skipBytes(length); - if (LOG.isTraceEnabled()) { - LOG.trace("Skipped Record with TxId {} DLSN {}", - currTxId, recordStream.getCurrentPosition()); - } + log.trace().attr("txId", currTxId).attr("dlsn", recordStream.getCurrentPosition()) + .log("Skipped Record"); recordStream.advance(1); } } catch (EOFException eof) { - if (LOG.isDebugEnabled()) { - LOG.debug("Skip encountered end of file Exception", eof); - } + log.debug().exception(eof).log("Skip encountered end of file Exception"); break; } } diff --git a/stream/distributedlog/protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java b/stream/distributedlog/protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java index a85920e68af..8bbb526bdc9 100644 --- a/stream/distributedlog/protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java +++ b/stream/distributedlog/protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java @@ -29,7 +29,7 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.concurrent.CompletableFuture; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.distributedlog.LogRecordSet.Reader; import org.apache.distributedlog.LogRecordSet.Writer; @@ -40,7 +40,7 @@ /** * Test Case for {@link LogRecordSet}. */ -@Slf4j +@CustomLog public class TestLogRecordSet { @Test(timeout = 60000) diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StandaloneStarter.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StandaloneStarter.java index 18484e63d8c..b0fb4bdb225 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StandaloneStarter.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StandaloneStarter.java @@ -23,7 +23,7 @@ import com.beust.jcommander.Parameter; import java.io.File; import java.util.concurrent.CountDownLatch; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.conf.ConfigurationUtil; import org.apache.bookkeeper.common.net.ServiceURI; import org.apache.commons.configuration2.CompositeConfiguration; @@ -32,7 +32,7 @@ /** * A starter to start a standalone cluster. */ -@Slf4j +@CustomLog public class StandaloneStarter { /** @@ -182,7 +182,7 @@ static int doMain(String[] args) throws Exception { try { liveLatch.await(); } catch (InterruptedException e) { - log.error("The standalone cluster is interrupted : ", e); + log.error().exception(e).log("The standalone cluster is interrupted"); } return 0; } diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java index c3657992a79..6b6d68be5ec 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java @@ -28,7 +28,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.stream.Collectors; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.StorageClientBuilder; import org.apache.bookkeeper.clients.admin.StorageAdminClient; import org.apache.bookkeeper.clients.config.StorageClientSettings; @@ -57,7 +57,7 @@ /** * A Cluster that runs a few storage nodes. */ -@Slf4j +@CustomLog public class StreamCluster extends AbstractLifecycleComponent { @@ -120,7 +120,9 @@ private void startZooKeeper() throws Exception { LocalDLMEmulator.runZookeeperOnAnyPort(spec.zkPort(), zkDir); zks = zkServerAndPort.getLeft(); zkPort = zkServerAndPort.getRight(); - log.info("Started zookeeper at port {}.", zkPort); + log.info() + .attr("zkPort", zkPort) + .log("Started zookeeper."); metadataServiceUri = ServiceURI.create( "zk://127.0.0.1:" + zkPort + "/ledgers"); } @@ -182,19 +184,24 @@ private LifecycleComponent startServer() throws Exception { StorageConfiguration storageConf = new StorageConfiguration(serverConf); storageConf.setRangeStoreDirNames(new String[]{rangesStoreDir.getPath()}); - log.info("Attempting to start storage server at (bookie port = {}, grpc port = {})" - + " : bkDir = {}, rangesStoreDir = {}", - bookiePort, grpcPort, bkDir, rangesStoreDir); + log.info() + .attr("bookiePort", bookiePort) + .attr("grpcPort", grpcPort) + .attr("bkDir", bkDir) + .attr("rangesStoreDir", rangesStoreDir) + .log("Attempting to start storage server"); server = StorageServer.buildStorageServer( serverConf, grpcPort); server.start(); - log.info("Started storage server at (bookie port = {}, grpc port = {})", - bookiePort, grpcPort); + log.info() + .attr("bookiePort", bookiePort) + .attr("grpcPort", grpcPort) + .log("Started storage server"); this.rpcEndpoints.add(StorageServer.createLocalEndpoint(grpcPort, false)); return server; } catch (Throwable e) { - log.error("Failed to start storage server", e); + log.error().exception(e).log("Failed to start storage server"); if (null != server) { server.stop(); } @@ -212,7 +219,9 @@ private LifecycleComponent startServer() throws Exception { } private void startServers() throws Exception { - log.info("Starting {} storage servers.", spec.numServers()); + log.info() + .attr("numServers", spec.numServers()) + .log("Starting storage servers."); ExecutorService executor = Executors.newCachedThreadPool(); List> startFutures = Lists.newArrayList(); for (int i = 0; i < spec.numServers(); i++) { @@ -222,7 +231,9 @@ private void startServers() throws Exception { for (Future future : startFutures) { servers.add(future.get()); } - log.info("Started {} storage servers.", spec.numServers()); + log.info() + .attr("numServers", spec.numServers()) + .log("Started storage servers."); executor.shutdown(); } @@ -237,7 +248,9 @@ private void createDefaultNamespaces() throws Exception { .serviceUri(serviceUri) .usePlaintext(true) .build(); - log.info("Service uri are : {}", serviceUri); + log.info() + .attr("serviceUri", serviceUri) + .log("Service uri are"); String namespaceName = "default"; try (StorageAdminClient admin = StorageClientBuilder.newBuilder() .withSettings(settings) @@ -247,11 +260,18 @@ private void createDefaultNamespaces() throws Exception { while (!created) { try { NamespaceProperties nsProps = result(admin.getNamespace(namespaceName)); - log.info("Namespace '{}':\n{}", namespaceName, nsProps); + log.info() + .attr("namespace", namespaceName) + .attr("properties", nsProps) + .log("Namespace properties"); created = true; } catch (NamespaceNotFoundException nnfe) { - log.info("Namespace '{}' is not found.", namespaceName); - log.info("Creating namespace '{}' ...", namespaceName); + log.info() + .attr("namespace", namespaceName) + .log("Namespace is not found."); + log.info() + .attr("namespace", namespaceName) + .log("Creating namespace ..."); try { NamespaceProperties nsProps = result( admin.createNamespace( @@ -259,8 +279,12 @@ private void createDefaultNamespaces() throws Exception { NamespaceConfiguration.newBuilder() .setDefaultStreamConf(DEFAULT_STREAM_CONF) .build())); - log.info("Successfully created namespace '{}':", namespaceName); - log.info("{}", nsProps); + log.info() + .attr("namespace", namespaceName) + .log("Successfully created namespace"); + log.info() + .attr("properties", nsProps) + .log("Namespace properties"); } catch (ClientException ce) { // encountered exception, try to fetch the namespace again } diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/StorageServer.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/StorageServer.java index 2024a0ae458..3132f20f0e3 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/StorageServer.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/StorageServer.java @@ -25,7 +25,7 @@ import java.util.concurrent.ExecutionException; import java.util.function.Supplier; import java.util.stream.Collectors; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.bookie.BookieResources; import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.bookkeeper.clients.impl.channel.StorageServerChannel; @@ -80,7 +80,7 @@ /** * A storage server is a server that run storage service and serving rpc requests. */ -@Slf4j +@CustomLog public class StorageServer { private static class ServerArguments { @@ -105,10 +105,15 @@ private static void loadConfFile(CompositeConfiguration conf, String confFile) Configuration loadedConf = ConfigurationUtil.newConfiguration(c -> c.propertiesBuilder(confFile)); conf.addConfiguration(loadedConf); } catch (ConfigurationException e) { - log.error("Malformed configuration file {}", confFile, e); + log.error() + .attr("confFile", confFile) + .exception(e) + .log("Malformed configuration file"); throw new IllegalArgumentException("Malformed configuration file " + confFile, e); } - log.info("Loaded configuration file {}", confFile); + log.info() + .attr("confFile", confFile) + .log("Loaded configuration file"); } public static Endpoint createLocalEndpoint(int port, boolean useHostname) throws UnknownHostException { @@ -120,7 +125,9 @@ public static Endpoint createLocalEndpoint(int port, boolean useHostname) throws hostname = InetAddress.getLocalHost().getHostAddress(); } - log.warn("Decided to use hostname {}", hostname); + log.warn() + .attr("hostname", hostname) + .log("Decided to use hostname"); return Endpoint.newBuilder() .setHostname(hostname) .setPort(port) @@ -135,7 +142,10 @@ public static void main(String[] args) { static int doMain(String[] args) { // register thread uncaught exception handler Thread.setDefaultUncaughtExceptionHandler((thread, exception) -> - log.error("Uncaught exception in thread {}: {}", thread.getName(), exception.getMessage())); + log.error() + .attr("thread", thread.getName()) + .exceptionMessage(exception) + .log("Uncaught exception in thread")); // parse the commandline ServerArguments arguments = new ServerArguments(); @@ -163,7 +173,7 @@ static int doMain(String[] args) { grpcPort, grpcUseHostname); } catch (Exception e) { - log.error("Invalid storage configuration", e); + log.error().exception(e).log("Invalid storage configuration"); System.err.println(e.getMessage()); return ExitCode.INVALID_CONF.code(); } @@ -237,16 +247,24 @@ public static LifecycleComponent buildStorageServer(CompositeConfiguration conf, statsProviderService = new StatsProviderService(bkConf); rootStatsLogger = statsProviderService.getStatsProvider().getStatsLogger(""); serverBuilder.addComponent(statsProviderService); - log.info("Bookie configuration : {}", bkConf.asJson()); + log.info() + .attr("bookieConfig", bkConf.asJson()) + .log("Bookie configuration"); } else { rootStatsLogger = checkNotNull(externalStatsLogger, "External stats logger is not provided while not starting stats provider"); } // dump configurations - log.info("Dlog configuration : {}", dlConf.asJson()); - log.info("Storage configuration : {}", storageConf.asJson()); - log.info("Server configuration : {}", serverConf.asJson()); + log.info() + .attr("dlogConfig", dlConf.asJson()) + .log("Dlog configuration"); + log.info() + .attr("storageConfig", storageConf.asJson()) + .log("Storage configuration"); + log.info() + .attr("serverConfig", serverConf.asJson()) + .log("Server configuration"); // Create the bookie service ServerConfiguration bkServerConf; @@ -276,7 +294,9 @@ public static LifecycleComponent buildStorageServer(CompositeConfiguration conf, new org.apache.bookkeeper.server.conf.BookieConfiguration(bkServerConf), rootStatsLogger); serverBuilder.addComponent(httpService); - log.info("Load lifecycle component : {}", HttpService.class.getName()); + log.info() + .attr("component", HttpService.class.getName()) + .log("Load lifecycle component"); } } else { diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcServer.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcServer.java index 1ef26423a5e..e84759cfaea 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcServer.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcServer.java @@ -22,7 +22,7 @@ import io.grpc.ServerServiceDefinition; import io.grpc.inprocess.InProcessServerBuilder; import java.io.IOException; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.component.AbstractLifecycleComponent; import org.apache.bookkeeper.common.grpc.proxy.ProxyHandlerRegistry; import org.apache.bookkeeper.common.grpc.stats.MonitoringServerInterceptor; @@ -36,7 +36,7 @@ /** * KeyRange Server. */ -@Slf4j +@CustomLog public class GrpcServer extends AbstractLifecycleComponent { public static GrpcServer build(GrpcServerSpec spec) { @@ -109,7 +109,7 @@ protected void doStart() { try { grpcServer.start(); } catch (IOException e) { - log.error("Failed to start grpc server", e); + log.error().exception(e).log("Failed to start grpc server"); throw new StorageServerRuntimeException("Failed to start grpc server", e); } } diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcStorageContainerService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcStorageContainerService.java index 8b6a26aab5d..bffbbb0a7fa 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcStorageContainerService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcStorageContainerService.java @@ -15,7 +15,7 @@ package org.apache.bookkeeper.stream.server.grpc; import io.grpc.stub.StreamObserver; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.stream.proto.common.Endpoint; import org.apache.bookkeeper.stream.proto.storage.GetStorageContainerEndpointRequest; import org.apache.bookkeeper.stream.proto.storage.GetStorageContainerEndpointResponse; @@ -28,7 +28,7 @@ /** * Grpc based storage container service. */ -@Slf4j +@CustomLog class GrpcStorageContainerService extends StorageContainerServiceImplBase { private final StorageContainerStore storageContainerStore; diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java index 4fd3154a287..d307ccbfd5b 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java @@ -24,9 +24,9 @@ import java.util.Arrays; import java.util.List; import java.util.function.Supplier; +import lombok.CustomLog; import lombok.Getter; import lombok.experimental.Accessors; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieImpl; import org.apache.bookkeeper.bookie.BookieResources; @@ -56,7 +56,7 @@ * a {@link org.apache.bookkeeper.proto.BookieServer}. */ @Accessors(fluent = true) -@Slf4j +@CustomLog public class BookieService extends AbstractLifecycleComponent { @Getter @@ -146,14 +146,14 @@ protected void doStop() { try { ledgerManager.close(); } catch (Exception e) { - log.error("Error shutting down ledger manager", e); + log.error().exception(e).log("Error shutting down ledger manager"); } } if (lmFactory != null) { try { lmFactory.close(); } catch (Exception e) { - log.error("Error shutting down ledger manager factory", e); + log.error().exception(e).log("Error shutting down ledger manager factory"); } } if (null != metadataDriver) { @@ -185,7 +185,7 @@ public void publishInfo(ComponentInfoPublisher componentInfoPublisher) { componentInfoPublisher.publishEndpoint(endpoint); } catch (UnknownHostException err) { - log.error("Cannot compute local address", err); + log.error().exception(err).log("Cannot compute local address"); } } } diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieWatchService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieWatchService.java index fb9f3833df3..1edca443b98 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieWatchService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieWatchService.java @@ -25,7 +25,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import lombok.Cleanup; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.component.AbstractLifecycleComponent; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.conf.ClientConfiguration; @@ -39,7 +39,7 @@ /** * A service that watches bookies and wait for minimum number of bookies to be alive. */ -@Slf4j +@CustomLog public class BookieWatchService extends AbstractLifecycleComponent { @@ -64,7 +64,9 @@ protected void doStart() { try { waitingForNumBookies(clientDriver.getRegistrationClient(), minNumBookies); } catch (Exception e) { - log.error("Encountered exceptions on waiting {} bookies to be alive", minNumBookies); + log.error() + .attr("minNumBookies", minNumBookies) + .log("Encountered exceptions on waiting for bookies to be alive"); throw new RuntimeException("Encountered exceptions on waiting " + minNumBookies + " bookies to be alive", e); } @@ -81,9 +83,11 @@ private static void waitingForNumBookies(RegistrationClient client, int minNumBo while (bookies.size() < minNumBookies) { TimeUnit.SECONDS.sleep(1); bookies = FutureUtils.result(client.getWritableBookies()).getValue(); - log.info("Only {} bookies are live since {} seconds elapsed, " - + "wait for another {} bookies for another 1 second", - bookies.size(), stopwatch.elapsed(TimeUnit.SECONDS), minNumBookies - bookies.size()); + log.info() + .attr("aliveBookies", bookies.size()) + .attr("elapsedSeconds", stopwatch.elapsed(TimeUnit.SECONDS)) + .attr("waitingForBookies", minNumBookies - bookies.size()) + .log("Waiting for additional bookies to become alive for another 1 second"); } } diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/ClusterControllerService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/ClusterControllerService.java index 8e451270415..3bfbf1c472c 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/ClusterControllerService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/ClusterControllerService.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.util.function.Supplier; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.component.AbstractLifecycleComponent; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stream.storage.api.cluster.ClusterController; @@ -29,7 +29,7 @@ /** * A service that runs cluster controller. */ -@Slf4j +@CustomLog public class ClusterControllerService extends AbstractLifecycleComponent { diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/CuratorProviderService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/CuratorProviderService.java index 9bd6ab4292d..166705a93e1 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/CuratorProviderService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/CuratorProviderService.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.util.function.Supplier; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.component.AbstractLifecycleComponent; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; @@ -34,7 +34,7 @@ /** * A service to provide a curator client. */ -@Slf4j +@CustomLog public class CuratorProviderService extends AbstractLifecycleComponent implements Supplier { @@ -59,7 +59,9 @@ public CuratorProviderService(ServerConfiguration bkServerConf, @Override protected void doStart() { curatorClient.start(); - log.info("Provided curator clients to zookeeper {}.", zkServers); + log.info() + .attr("zkServers", zkServers) + .log("Provided curator clients to zookeeper."); } @Override diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/DLNamespaceProviderService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/DLNamespaceProviderService.java index 0a87add7888..950795d276d 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/DLNamespaceProviderService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/DLNamespaceProviderService.java @@ -16,8 +16,8 @@ import java.io.IOException; import java.net.URI; import java.util.function.Supplier; +import lombok.CustomLog; import lombok.Getter; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.component.AbstractLifecycleComponent; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; @@ -39,7 +39,7 @@ * *

    TODO: eliminate the direct usage of zookeeper here {@link https://github.com/apache/bookkeeper/issues/1331} */ -@Slf4j +@CustomLog public class DLNamespaceProviderService extends AbstractLifecycleComponent implements Supplier { @@ -52,17 +52,24 @@ private static URI initializeNamespace(ServerConfiguration bkServerConf, DLMetadata dlMetadata = DLMetadata.create(dlConfig); try { - log.info("Initializing dlog namespace at {}", dlogUri); + log.info() + .attr("dlogUri", dlogUri) + .log("Initializing dlog namespace"); dlMetadata.create(dlogUri); - log.info("Initialized dlog namespace at {}", dlogUri); + log.info() + .attr("dlogUri", dlogUri) + .log("Initialized dlog namespace"); } catch (ZKException e) { if (e.getKeeperExceptionCode() == Code.NODEEXISTS) { - if (log.isDebugEnabled()) { - log.debug("Dlog uri is already bound at {}", dlogUri); - } + log.debug() + .attr("dlogUri", dlogUri) + .log("Dlog uri is already bound"); return dlogUri; } - log.error("Failed to initialize dlog namespace at {}", dlogUri, e); + log.error() + .attr("dlogUri", dlogUri) + .exception(e) + .log("Failed to initialize dlog namespace"); throw e; } return dlogUri; @@ -122,7 +129,9 @@ protected void doStart() { throw new RuntimeException("Failed to build the distributedlog namespace at " + bkServerConf.getMetadataServiceUriUnchecked(), e); } - log.info("Provided distributedlog namespace at {}.", uri); + log.info() + .attr("uri", uri) + .log("Provided distributedlog namespace."); } @Override diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationServiceProvider.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationServiceProvider.java index ed31c90caf9..aabb88e20a7 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationServiceProvider.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationServiceProvider.java @@ -27,8 +27,8 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.function.Supplier; +import lombok.CustomLog; import lombok.SneakyThrows; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.component.AbstractLifecycleComponent; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -44,7 +44,7 @@ /** * A service that is responsible for registration using bookkeeper registration api. */ -@Slf4j +@CustomLog public class RegistrationServiceProvider extends AbstractLifecycleComponent implements Supplier { @@ -99,10 +99,16 @@ protected void doStart() { .build(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - log.error("Interrupted at creating zookeeper client to {}", zkServers, e); + log.error() + .attr("zkServers", zkServers) + .exception(e) + .log("Interrupted at creating zookeeper client"); throw e; } catch (Exception e) { - log.error("Failed to create zookeeper client to {}", zkServers, e); + log.error() + .attr("zkServers", zkServers) + .exception(e) + .log("Failed to create zookeeper client"); throw e; } client = new ZKRegistrationClient(zkClient, regPath, regExecutor, bookieAddresschangeTracking); @@ -125,7 +131,10 @@ protected void doClose() throws IOException { zkClient.close(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - log.warn("Interrupted at closing zookeeper client to {}", zkServers, e); + log.warn() + .attr("zkServers", zkServers) + .exception(e) + .log("Interrupted at closing zookeeper client"); } } this.regExecutor.shutdown(); diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java index 8d5ad8b0074..9724e17c27c 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/RegistrationStateService.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.util.Collections; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.bookie.BookieStateManager; import org.apache.bookkeeper.clients.utils.NetUtils; import org.apache.bookkeeper.common.component.AbstractLifecycleComponent; @@ -38,7 +38,7 @@ * *

    It registers the server to registration service and handle the state transition. */ -@Slf4j +@CustomLog public class RegistrationStateService extends AbstractLifecycleComponent { @@ -85,8 +85,10 @@ protected void doStart() { BookieServiceInfo.NO_INFO); stateManager.initState(); stateManager.registerBookie(true).get(); - log.info("Successfully register myself under registration path {}/{}", - regServiceProvider.getRegistrationPath(), NetUtils.endpointToString(myEndpoint)); + log.info() + .attr("registrationPath", regServiceProvider.getRegistrationPath()) + .attr("endpoint", NetUtils.endpointToString(myEndpoint)) + .log("Successfully register myself under registration path"); } catch (Exception e) { throw new RuntimeException("Failed to initialize a registration state service", e); } diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/StorageService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/StorageService.java index 4aedd4eeda6..f40ec599e19 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/StorageService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/StorageService.java @@ -16,7 +16,7 @@ import java.io.IOException; import java.util.function.Supplier; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.component.AbstractLifecycleComponent; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stream.storage.StorageContainerStoreBuilder; @@ -26,7 +26,7 @@ /** * Service to run the storage {@link StorageContainerStore}. */ -@Slf4j +@CustomLog public class StorageService extends AbstractLifecycleComponent implements Supplier { diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/StateStores.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/StateStores.java index 1d7d11265a1..f67cf21d0af 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/StateStores.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/StateStores.java @@ -20,8 +20,8 @@ import java.util.function.Supplier; import lombok.AccessLevel; +import lombok.CustomLog; import lombok.NoArgsConstructor; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public; import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving; import org.apache.bookkeeper.statelib.api.kv.KVAsyncStore; @@ -37,7 +37,7 @@ */ @Public @Evolving -@Slf4j +@CustomLog @NoArgsConstructor(access = AccessLevel.PRIVATE) public final class StateStores { diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/journal/AbstractStateStoreWithJournal.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/journal/AbstractStateStoreWithJournal.java index 58977f3b630..a93353d1faf 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/journal/AbstractStateStoreWithJournal.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/journal/AbstractStateStoreWithJournal.java @@ -33,8 +33,8 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; +import lombok.CustomLog; import lombok.Getter; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.statelib.api.AsyncStateStore; @@ -60,7 +60,7 @@ /** * An abstract implementation of {@link AsyncStateStore} with journal. */ -@Slf4j +@CustomLog public abstract class AbstractStateStoreWithJournal implements AsyncStateStore { // local state store instance @@ -161,7 +161,7 @@ public CompletableFuture init(StateStoreSpec spec) { try { validateStoreSpec(spec); } catch (IllegalArgumentException e) { - log.error("Fail to init state store due to : ", e); + log.error().exception(e).log("Fail to init state store"); return FutureUtils.exception(e); } this.spec = spec; @@ -205,7 +205,8 @@ public CompletableFuture init(StateStoreSpec spec) { return initializeLocalStore(spec) .thenComposeAsync(ignored -> initializeJournalWriter(spec), writeIOScheduler) .thenComposeAsync(endDLSN -> { - log.info("Successfully write a barrier record for mvcc store {} at {}", name, endDLSN); + log.info().attr("name", name).attr("endDLSN", endDLSN) + .log("Successfully write a barrier record for mvcc store"); return replayJournal(endDLSN); }, writeIOScheduler); } @@ -213,9 +214,9 @@ public CompletableFuture init(StateStoreSpec spec) { private CompletableFuture initializeLocalStore(StateStoreSpec spec) { return executeWriteIO(() -> { - log.info("Initializing the local state for mvcc store {}", name()); + log.info().attr("name", name()).log("Initializing the local state for mvcc store"); localStore.init(spec); - log.info("Initialized the local state for mvcc store {}", name()); + log.info().attr("name", name()).log("Initialized the local state for mvcc store"); commandProcessor = newCommandProcessor(); return null; }); @@ -243,8 +244,8 @@ private CompletableFuture initializeJournalWriter(StateStoreSpec spec) { if (nextRevision < 0) { nextRevision = 0L; } - log.info("Initialized the journal writer for mvcc store {} : last revision = {}", - name(), nextRevision); + log.info().attr("name", name()).attr("lastRevision", nextRevision) + .log("Initialized the journal writer for mvcc store"); } return writeCommandBuf(newCatchupMarker()); }, writeIOScheduler); @@ -261,8 +262,8 @@ private CompletableFuture writeCatchUpMarker() { if (nextRevision < 0) { nextRevision = 0L; } - log.info("Initialized the journal writer for writing catchup marker to mvcc store {} :" - + " last revision = {}", name(), nextRevision); + log.info().attr("name", name()).attr("lastRevision", nextRevision) + .log("Initialized the journal writer for writing catchup marker to mvcc store"); } return writeCommandBuf(newCatchupMarker()); }).thenCompose(dlsn -> { @@ -322,7 +323,8 @@ private CompletableFuture replayJournal(DLSN endDLSN) { return null; }); - log.info("Successfully open the journal reader for mvcc store {} : end dlsn = {}", name(), endDLSN); + log.info().attr("name", name()).attr("endDLSN", endDLSN) + .log("Successfully open the journal reader for mvcc store"); replayJournal(r, endDLSN, replayFuture, lastRevision); return replayFuture; }, writeIOScheduler); @@ -376,33 +378,35 @@ private FutureEventListener newRecordHandler(AsyncLogReader r return new FutureEventListener() { @Override public void onSuccess(LogRecordWithDLSN record) { - if (log.isDebugEnabled()) { - log.debug("Received command record {} @ {} to replay at mvcc store {}", - record, record.getDlsn(), name()); - } + log.debug().attr("record", record) + .attr("dlsn", record.getDlsn()) + .attr("name", name()) + .log("Received command record to replay at mvcc store"); try { - if (log.isDebugEnabled()) { - log.debug("Applying command transaction {} - record {} @ {} to mvcc store {}", - record.getTransactionId(), record, record.getDlsn(), name()); - } + log.debug().attr("txid", record.getTransactionId()) + .attr("record", record) + .attr("dlsn", record.getDlsn()) + .attr("name", name()) + .log("Applying command transaction to mvcc store"); commandProcessor.applyCommand(record.getTransactionId(), record.getPayloadBuf(), localStore); if (record.getDlsn().compareTo(endDLSN) >= 0) { - log.info("Finished replaying journal for state store {}", name()); + log.info().attr("name", name()).log("Finished replaying journal for state store"); markInitialized(reader); FutureUtils.complete(future, null); return; } - if (log.isDebugEnabled()) { - log.debug("Read next record after {} at mvcc store {}", - record.getDlsn(), name()); - } + log.debug().attr("dlsn", record.getDlsn()) + .attr("name", name()) + .log("Read next record at mvcc store"); // read next record replayJournal(reader, endDLSN, future); } catch (Exception e) { - log.error("Exception is thrown when applying command record {} @ {} to mvcc store {}", - record, record.getDlsn(), name()); + log.error().attr("record", record) + .attr("dlsn", record.getDlsn()) + .attr("name", name()) + .log("Exception is thrown when applying command record to mvcc store"); FutureUtils.completeExceptionally(future, e); } } @@ -425,16 +429,16 @@ private void replayLoop(AsyncLogReader reader) { reader.readNext().whenComplete(new FutureEventListener() { @Override public void onSuccess(LogRecordWithDLSN record) { - if (log.isDebugEnabled()) { - log.debug("Received command record {} @ {} to replay at mvcc store {}", - record, record.getDlsn(), name()); - } + log.debug().attr("record", record) + .attr("dlsn", record.getDlsn()) + .attr("name", name()) + .log("Received command record to replay at mvcc store"); try { commandProcessor.applyCommand(record.getTransactionId(), record.getPayloadBuf(), localStore); // read next record replayLoop(reader); } catch (StateStoreRuntimeException e) { - log.error("Fail to reply command record {}", record, e); + log.error().attr("record", record).exception(e).log("Fail to reply command record"); // TODO: handle state store exception } } @@ -459,12 +463,12 @@ public CompletableFuture closeAsync() { // cancel checkpoint task if (null != checkpointTask) { if (!checkpointTask.cancel(true)) { - log.warn("Fail to cancel checkpoint task of state store {}", name()); + log.warn().attr("name", name()).log("Fail to cancel checkpoint task of state store"); } } // wait until last checkpoint task completed writeIOScheduler.submit(() -> { - log.info("closing async state store {}", name); + log.info().attr("name", name).log("closing async state store"); FutureUtils.ensure( // close the log streams Utils.closeSequence( @@ -473,7 +477,7 @@ public CompletableFuture closeAsync() { getWriter(), logManager ).thenRun(() -> { - log.info("Successfully close the log stream of state store {}", name); + log.info().attr("name", name).log("Successfully close the log stream of state store"); }), // close the local state store () -> { @@ -507,7 +511,7 @@ private void closeLocalStore() { try { localStore.flush(); } catch (StateStoreException e) { - log.warn("Fail to flush local state store {}", name(), e); + log.warn().attr("name", name()).exception(e).log("Fail to flush local state store"); } // close the local store localStore.close(); diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/KVCommandProcessor.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/KVCommandProcessor.java index 1820c8326fc..1eb9009a4c5 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/KVCommandProcessor.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/kv/KVCommandProcessor.java @@ -25,8 +25,8 @@ import io.netty.buffer.ByteBufUtil; import lombok.AccessLevel; import lombok.Cleanup; +import lombok.CustomLog; import lombok.NoArgsConstructor; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.proto.statestore.kv.Command; import org.apache.bookkeeper.proto.statestore.kv.DeleteRequest; import org.apache.bookkeeper.proto.statestore.kv.PutRequest; @@ -35,7 +35,7 @@ /** * A command processor to apply commands happened to the kv state store. */ -@Slf4j +@CustomLog @NoArgsConstructor(access = AccessLevel.PRIVATE) class KVCommandProcessor implements CommandProcessor> { @@ -49,7 +49,8 @@ public void applyCommand(long revision, ByteBuf cmdBuf, RocksdbKVStore key type * @param value type */ -@Slf4j +@CustomLog public class RocksdbKVStore implements KVStore { private static final byte[] METADATA_CF = ".meta".getBytes(UTF_8); @@ -189,18 +189,19 @@ private void loadRocksdbFromCheckpointStore(StateStoreSpec spec) throws StateSto .forEach(cp -> cp.remove(localStorePath)); // delete everything else break; } catch (TimeoutException e) { - log.error("Timeout waiting for checkpoint restore: {}", cpi, e); + log.error().attr("checkpoint", cpi).exception(e).log("Timeout waiting for checkpoint restore"); throw new StateStoreException("Failed to restore checkpoint: " + cpi.getId(), e); } catch (StateStoreException e) { // Got an exception. Log and try the next checkpoint - log.error("Failed to restore checkpoint: {}", cpi, e); + log.error().attr("checkpoint", cpi).exception(e).log("Failed to restore checkpoint"); } } } @Override public synchronized void checkpoint() { - log.info("Checkpoint local state store {} at revision {}", name, getLastRevision()); + log.info().attr("name", name).attr("revision", getLastRevision()) + .log("Checkpoint local state store"); byte[] checkpointAtRevisionBytes = new byte[Long.BYTES]; System.arraycopy(lastRevisionBytes, 0, checkpointAtRevisionBytes, 0, checkpointAtRevisionBytes.length); checkpointScheduler.submit(() -> { @@ -208,8 +209,10 @@ public synchronized void checkpoint() { // TODO: move create checkpoint to the checkpoint method checkpointer.checkpointAtTxid(checkpointAtRevisionBytes); } catch (StateStoreException e) { - log.error("Failed to checkpoint state store {} at revision {}", - name, Bytes.toLong(checkpointAtRevisionBytes, 0), e); + log.error().attr("name", name) + .attr("revision", Bytes.toLong(checkpointAtRevisionBytes, 0)) + .exception(e) + .log("Failed to checkpoint state store"); } }); } @@ -413,10 +416,12 @@ protected Pair> openRocksdb( cfHandles); return Pair.of(db, cfHandles); } catch (IOException ioe) { - log.error("Failed to create parent directory {} for opening rocksdb", dir.getParentFile().toPath(), ioe); + log.error().attr("directory", dir.getParentFile().toPath()).exception(ioe) + .log("Failed to create parent directory for opening rocksdb"); throw new StateStoreException(ioe); } catch (RocksDBException dbe) { - log.error("Failed to open rocksdb at dir {}", dir.getAbsolutePath(), dbe); + log.error().attr("directory", dir.getAbsolutePath()).exception(dbe) + .log("Failed to open rocksdb"); throw new StateStoreException(dbe); } } @@ -465,7 +470,7 @@ private void cleanupLocalStoreDir(File dbDir) { try { MoreFiles.deleteRecursively(dbDir.toPath(), RecursiveDeleteOption.ALLOW_INSECURE); } catch (IOException e) { - log.error("Failed to cleanup localStoreDir", e); + log.error().exception(e).log("Failed to cleanup localStoreDir"); } } } diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCAsyncBytesStoreImpl.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCAsyncBytesStoreImpl.java index ae3c6ae665b..07114366310 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCAsyncBytesStoreImpl.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCAsyncBytesStoreImpl.java @@ -22,7 +22,7 @@ import io.netty.buffer.ByteBuf; import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.api.kv.op.DeleteOp; import org.apache.bookkeeper.api.kv.op.IncrementOp; import org.apache.bookkeeper.api.kv.op.OpFactory; @@ -45,7 +45,7 @@ /** * MVCC Async Store Implementation. */ -@Slf4j +@CustomLog class MVCCAsyncBytesStoreImpl extends AbstractStateStoreWithJournal> implements MVCCAsyncStore { diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCCommandProcessor.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCCommandProcessor.java index dc3e8059d82..9203ff9c49b 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCCommandProcessor.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCCommandProcessor.java @@ -20,8 +20,8 @@ import io.netty.buffer.ByteBuf; import lombok.AccessLevel; +import lombok.CustomLog; import lombok.NoArgsConstructor; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.api.kv.op.DeleteOp; import org.apache.bookkeeper.api.kv.op.IncrementOp; import org.apache.bookkeeper.api.kv.op.PutOp; @@ -39,7 +39,7 @@ import org.apache.bookkeeper.statelib.impl.mvcc.op.proto.ProtoTxnOpImpl; import org.apache.bookkeeper.stream.proto.kv.store.Command; -@Slf4j +@CustomLog @NoArgsConstructor(access = AccessLevel.PRIVATE) class MVCCCommandProcessor implements CommandProcessor> { @@ -102,10 +102,10 @@ private void applyTxnOp(long revision, boolean ignoreSmallerRevision, MVCCStoreImpl localStore) { try (TxnResult result = localStore.processTxn(revision, op)) { - if (log.isDebugEnabled()) { - log.debug("Result after applying transaction {} : {} - success = {}", - revision, result.code(), result.isSuccess()); - } + log.debug().attr("revision", revision) + .attr("code", result.code()) + .attr("success", result.isSuccess()) + .log("Result after applying transaction"); if (Code.OK == result.code() || (ignoreSmallerRevision && Code.SMALLER_REVISION == result.code())) { return; diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCStoreImpl.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCStoreImpl.java index d0ac0cb6620..450f6510123 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCStoreImpl.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCStoreImpl.java @@ -36,7 +36,7 @@ import java.util.List; import java.util.NoSuchElementException; import javax.annotation.Nullable; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.api.kv.impl.op.OpFactoryImpl; import org.apache.bookkeeper.api.kv.impl.result.DeleteResultImpl; import org.apache.bookkeeper.api.kv.impl.result.IncrementResultImpl; @@ -91,7 +91,7 @@ *

    The current implementation executes write operations in one single io thread. * It can be improved later to leverage the revision numbers to achieve mvcc. */ -@Slf4j +@CustomLog class MVCCStoreImpl extends RocksdbKVStore implements MVCCStore { private static final Comparator COMPARATOR = UnsignedBytes.lexicographicalComparator(); diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCUtils.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCUtils.java index ac96a9289b1..fa2a4f14125 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCUtils.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCUtils.java @@ -29,8 +29,8 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import lombok.AccessLevel; +import lombok.CustomLog; import lombok.NoArgsConstructor; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.api.kv.op.CompareOp; import org.apache.bookkeeper.api.kv.op.CompareResult; import org.apache.bookkeeper.api.kv.op.CompareTarget; @@ -61,7 +61,7 @@ /** * Utils for mvcc stores. */ -@Slf4j +@CustomLog @NoArgsConstructor(access = AccessLevel.PRIVATE) public final class MVCCUtils { @@ -306,7 +306,7 @@ static Command newCommand(ByteBuf recordBuf) { try { return Command.parseFrom(recordBuf.nioBuffer()); } catch (InvalidProtocolBufferException e) { - log.error("Found a corrupted record on replaying log stream", e); + log.error().exception(e).log("Found a corrupted record on replaying log stream"); throw new StateStoreRuntimeException("Found a corrupted record on replaying log stream", e); } } diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointFile.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointFile.java index 7e4be2a2ee4..8df0fda4250 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointFile.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointFile.java @@ -37,7 +37,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.statelib.api.checkpoint.CheckpointStore; import org.apache.bookkeeper.statelib.impl.rocksdb.RocksUtils; import org.apache.bookkeeper.stream.proto.kv.store.CheckpointMetadata; @@ -46,7 +46,7 @@ /** * CheckpointFile encapsulates the attributes and operations for a file in checkpoint. */ -@Slf4j +@CustomLog @lombok.Builder @lombok.EqualsAndHashCode public class CheckpointFile { @@ -84,7 +84,10 @@ private static String computeChecksum(File file) { ckSum = Files.asByteSource(file).hash(Hashing.sha256()).toString(); return ckSum; } catch (IOException e) { - log.error("Failed to get checksum for file {} {}", file.getName(), e.getMessage(), e); + log.error().attr("fileName", file.getName()) + .exceptionMessage(e) + .exception(e) + .log("Failed to get checksum for file"); } return ckSum; } @@ -169,7 +172,10 @@ public boolean needCopy(CheckpointStore checkpointStore, String dbPrefix, boolea return false; } } catch (IOException e) { - log.error("Failed fileExists {} {}", file.getName(), e.getMessage(), e); + log.error().attr("fileName", file.getName()) + .exceptionMessage(e) + .exception(e) + .log("Failed fileExists"); } return true; } @@ -242,8 +248,11 @@ public void copyFromRemote(CheckpointStore checkpointStore, // Check if we made any progress long endMs = System.currentTimeMillis(); long newCount = cis.getCount(); - log.info("Timeout waiting for copy: {} last-read {} current-read {} runtime(ms) {} ", - remoteFilePath, lastCount, newCount, endMs - startMs); + log.info().attr("remoteFilePath", remoteFilePath) + .attr("lastRead", lastCount) + .attr("currentRead", newCount) + .attr("runtimeMs", endMs - startMs) + .log("Timeout waiting for copy"); if (lastCount == newCount) { throw new TimeoutException("No progress reading: " + remoteFilePath + " read " + lastCount + " runtime(ms) " + (endMs - startMs) diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointInfo.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointInfo.java index f38bf28b388..b598a1f227e 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointInfo.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointInfo.java @@ -31,7 +31,7 @@ import java.time.Duration; import java.util.UUID; import java.util.concurrent.TimeoutException; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.statelib.api.checkpoint.CheckpointStore; import org.apache.bookkeeper.statelib.api.exceptions.StateStoreException; import org.apache.bookkeeper.stream.proto.kv.store.CheckpointMetadata; @@ -40,7 +40,7 @@ /** * CheckpointInfo encapsulated information and operation for a checkpoint. */ -@Slf4j +@CustomLog public class CheckpointInfo implements Comparable { final String id; @@ -96,8 +96,10 @@ public void remove(File dbPath) { MoreFiles.deleteRecursively(checkpointPath, RecursiveDeleteOption.ALLOW_INSECURE); } } catch (IOException ioe) { - log.warn("Failed to remove unused checkpoint {} from {}", - id, getCheckpointBaseDir(dbPath), ioe); + log.warn().attr("checkpointId", id) + .attr("directory", getCheckpointBaseDir(dbPath)) + .exception(ioe) + .log("Failed to remove unused checkpoint"); } } @@ -136,7 +138,9 @@ public CheckpointMetadata restore(File dbPath, RocksdbRestoreTask task) task.restore(id, metadata); updateCurrent(dbPath); - log.info("Successfully restore checkpoint {} to {}", id, getCheckpointPath(dbPath)); + log.info().attr("checkpointId", id) + .attr("path", getCheckpointPath(dbPath)) + .log("Successfully restore checkpoint"); return metadata; } public CheckpointMetadata restore(String dbName, File dbPath, CheckpointStore store) @@ -153,7 +157,7 @@ public CheckpointMetadata restore(String dbName, File dbPath, CheckpointStore st RocksdbRestoreTask task = new RocksdbRestoreTask(dbName, checkpointsDir, store, maxIdle); return restore(dbPath, task); } catch (IOException ioe) { - log.error("Failed to restore rocksdb {}", dbName, ioe); + log.error().attr("dbName", dbName).exception(ioe).log("Failed to restore rocksdb"); throw new StateStoreException("Failed to restore rocksdb " + dbName, ioe); } } diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointer.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointer.java index f5f507a5a8f..f1b90b97e1f 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointer.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointer.java @@ -28,7 +28,7 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.TimeoutException; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.statelib.api.checkpoint.CheckpointStore; import org.apache.bookkeeper.statelib.api.exceptions.StateStoreException; import org.apache.bookkeeper.statelib.impl.rocksdb.RocksUtils; @@ -39,7 +39,7 @@ /** * Rocksdb Checkpointer that manages checkpoints. */ -@Slf4j +@CustomLog public class RocksCheckpointer implements AutoCloseable { public static CheckpointMetadata restore(String dbName, @@ -74,8 +74,10 @@ private static void cleanupLocalCheckpoints(File dbPath, String checkpointToExcl Paths.get(checkpointsDir.getAbsolutePath(), checkpoint), RecursiveDeleteOption.ALLOW_INSECURE); } catch (IOException ioe) { - log.warn("Failed to remove unused checkpoint {} from {}", - checkpoint, checkpointsDir, ioe); + log.warn().attr("checkpoint", checkpoint) + .attr("directory", checkpointsDir) + .exception(ioe) + .log("Failed to remove unused checkpoint"); } } } @@ -99,7 +101,7 @@ public static List getCheckpoints(String dbPrefix, CheckpointSto try { files = store.listFiles(remoteCheckpointsPath); } catch (IOException e) { - log.warn("No remote checkpoints available. Starting with nullCheckpoint", e); + log.warn().exception(e).log("No remote checkpoints available. Starting with nullCheckpoint"); return result; } @@ -111,9 +113,11 @@ public static List getCheckpoints(String dbPrefix, CheckpointSto try (InputStream is = store.openInputStream(metadataPath)) { result.add(new CheckpointInfo(checkpointId, is)); } catch (FileNotFoundException fnfe) { - log.error("Metadata is corrupt for the checkpoint {}. Skipping it.", checkpointId); + log.error() + .attr("checkpointId", checkpointId) + .log("Metadata is corrupt for the checkpoint. Skipping it."); } catch (IOException e) { - log.error("IO exception {}, Skipping it", checkpointId, e); + log.error().attr("checkpointId", checkpointId).exception(e).log("IO exception, Skipping it"); } } Collections.sort(result, Collections.reverseOrder()); diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksdbCheckpointTask.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksdbCheckpointTask.java index 99b2a97f58c..6e1d5be0d5f 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksdbCheckpointTask.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksdbCheckpointTask.java @@ -29,7 +29,7 @@ import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.statelib.api.checkpoint.CheckpointStore; import org.apache.bookkeeper.statelib.api.exceptions.StateStoreException; import org.apache.bookkeeper.statelib.impl.rocksdb.RocksUtils; @@ -40,7 +40,7 @@ /** * A task that periodically checkpoints rocksdb instance. */ -@Slf4j +@CustomLog public class RocksdbCheckpointTask { @@ -93,8 +93,8 @@ public String checkpoint(byte[] txid) throws StateStoreException { String checkpointId = UUID.randomUUID().toString(); File tempDir = new File(checkpointDir, checkpointId); - log.info("Create a local checkpoint of state store {} at {}", - dbName, tempDir); + log.info().attr("dbName", dbName).attr("directory", tempDir) + .log("Create a local checkpoint of state store"); try { try { checkpoint.createCheckpoint(tempDir.getAbsolutePath()); @@ -134,7 +134,8 @@ public String checkpoint(byte[] txid) throws StateStoreException { return checkpointId; } catch (IOException ioe) { - log.error("Failed to checkpoint db {} to dir {}", new Object[] { dbName, tempDir, ioe }); + log.error().attr("dbName", dbName).attr("directory", tempDir).exception(ioe) + .log("Failed to checkpoint db"); throw new StateStoreException( "Failed to checkpoint db " + dbName + " to dir " + tempDir, ioe); @@ -145,7 +146,8 @@ public String checkpoint(byte[] txid) throws StateStoreException { Paths.get(tempDir.getAbsolutePath()), RecursiveDeleteOption.ALLOW_INSECURE); } catch (IOException ioe) { - log.warn("Failed to remove temporary checkpoint dir {}", tempDir, ioe); + log.warn().attr("directory", tempDir).exception(ioe) + .log("Failed to remove temporary checkpoint dir"); } } } @@ -213,8 +215,8 @@ private void cleanupRemoteCheckpoints(File checkpointedDir, String remoteCheckpointPath = RocksUtils.getDestCheckpointPath(dbPrefix, checkpoint); checkpointStore.deleteRecursively( remoteCheckpointPath); - log.info("Delete remote checkpoint {} from checkpoint store at {}", - checkpoint, remoteCheckpointPath); + log.info().attr("checkpoint", checkpoint).attr("path", remoteCheckpointPath) + .log("Delete remote checkpoint from checkpoint store"); } // delete unused ssts diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksdbRestoreTask.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksdbRestoreTask.java index 70dd2e3e76e..7d670c47475 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksdbRestoreTask.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksdbRestoreTask.java @@ -24,7 +24,7 @@ import java.time.Duration; import java.util.List; import java.util.concurrent.TimeoutException; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.statelib.api.checkpoint.CheckpointStore; import org.apache.bookkeeper.statelib.api.exceptions.StateStoreException; import org.apache.bookkeeper.stream.proto.kv.store.CheckpointMetadata; @@ -32,7 +32,7 @@ /** * A task that restore a rocksdb instance. */ -@Slf4j +@CustomLog public class RocksdbRestoreTask { private final String dbName; @@ -70,8 +70,10 @@ public void restore(String checkpointId, CheckpointMetadata metadata) throws Sta List files = getCheckpointFiles(checkpointedDir, metadata); copyFilesFromRemote(checkpointId, files); } catch (IOException ioe) { - log.error("Failed to restore checkpoint {} to local directory {}", - new Object[] { checkpointId, checkpointedDir, ioe }); + log.error().attr("checkpointId", checkpointId) + .attr("directory", checkpointedDir) + .exception(ioe) + .log("Failed to restore checkpoint to local directory"); throw new StateStoreException( "Failed to restore checkpoint " + checkpointId + " to local directory " + checkpointedDir, ioe); diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLCheckpointStore.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLCheckpointStore.java index 7046ef5860d..181a24b3f32 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLCheckpointStore.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLCheckpointStore.java @@ -30,7 +30,7 @@ import java.util.Iterator; import java.util.List; import java.util.concurrent.ExecutionException; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.statelib.api.checkpoint.CheckpointStore; import org.apache.bookkeeper.statelib.impl.Constants; import org.apache.distributedlog.DLSN; @@ -48,7 +48,7 @@ /** * Dlog based checkpoint store. */ -@Slf4j +@CustomLog public class DLCheckpointStore implements CheckpointStore { private final Namespace namespace; @@ -119,7 +119,7 @@ public OutputStream openOutputStream(String filePath) throws IOException { @Override public void rename(String srcLog, String dstLog) throws IOException { - log.info("Renaming {} to {}", srcLog, dstLog); + log.info().attr("srcLog", srcLog).attr("dstLog", dstLog).log("Renaming"); try { namespace.renameLog(srcLog, dstLog).get(); } catch (InterruptedException e) { diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLInputStream.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLInputStream.java index c9e5fa8d6f5..07c560bdb96 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLInputStream.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLInputStream.java @@ -22,7 +22,7 @@ import java.io.IOException; import java.io.InputStream; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.distributedlog.LogRecordWithDLSN; import org.apache.distributedlog.api.DistributedLogManager; import org.apache.distributedlog.api.LogReader; @@ -30,7 +30,7 @@ /** * The input stream for a distributedlog stream. */ -@Slf4j +@CustomLog class DLInputStream extends InputStream { private static final long REOPEN_READER_SKIP_BYTES = 4 * 1024 * 1024; // 4MB diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLOutputStream.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLOutputStream.java index 8b229dfaa6f..db5f04c36c1 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLOutputStream.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/dlog/DLOutputStream.java @@ -26,7 +26,7 @@ import java.io.OutputStream; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.distributedlog.DLSN; @@ -39,7 +39,7 @@ /** * DistributedLog Output Stream. */ -@Slf4j +@CustomLog class DLOutputStream extends OutputStream { private static final byte[] CONTROL_RECORD_CONTENT = "control".getBytes(UTF_8); @@ -127,7 +127,7 @@ public void flush() throws IOException { } catch (IOException ioe) { throw ioe; } catch (Exception e) { - log.error("Unexpected exception in DLOutputStream", e); + log.error().exception(e).log("Unexpected exception in DLOutputStream"); throw new UnexpectedException("unexpected exception in DLOutputStream#flush()", e); } } diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStore.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStore.java index 8b29b1ceb97..6c3d954f253 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStore.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStore.java @@ -26,7 +26,7 @@ import java.io.File; import java.net.URI; import java.nio.file.Files; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.coder.ByteArrayCoder; import org.apache.bookkeeper.statelib.api.StateStoreSpec; import org.apache.commons.io.FileUtils; @@ -43,7 +43,7 @@ /** * Unit test of {@link RocksdbKVStore}. */ -@Slf4j +@CustomLog public class TestRocksdbKVAsyncStore extends TestDistributedLogBase { private static URI uri; diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStoreWithCheckpoints.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStoreWithCheckpoints.java index 904e8531243..e4fdc51e333 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStoreWithCheckpoints.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVAsyncStoreWithCheckpoints.java @@ -38,7 +38,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.coder.ByteArrayCoder; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.statelib.api.StateStoreSpec; @@ -64,7 +64,7 @@ /** * Unit test of {@link RocksdbKVStore}. */ -@Slf4j +@CustomLog public class TestRocksdbKVAsyncStoreWithCheckpoints extends TestDistributedLogBase { private static URI uri; diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVStoreCheckpoint.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVStoreCheckpoint.java index d11ffa76ce6..b55856b232a 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVStoreCheckpoint.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestRocksdbKVStoreCheckpoint.java @@ -24,7 +24,7 @@ import static org.junit.Assert.assertNull; import java.io.File; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.statelib.impl.rocksdb.checkpoint.CheckpointInfo; import org.junit.After; import org.junit.Before; @@ -39,7 +39,7 @@ /** * Test cases for Rocksdb KV Store with checkpoints. */ -@Slf4j +@CustomLog public class TestRocksdbKVStoreCheckpoint { @Rule diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestStateStore.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestStateStore.java index c4a5b23002b..a21f1af104b 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestStateStore.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/kv/TestStateStore.java @@ -32,7 +32,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeoutException; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.coder.StringUtf8Coder; import org.apache.bookkeeper.statelib.api.StateStoreSpec; import org.apache.bookkeeper.statelib.api.checkpoint.CheckpointStore; @@ -51,7 +51,7 @@ /** * TestStateStore is a helper class for testing various statestore operations. */ -@Slf4j +@CustomLog public class TestStateStore { private final String dbName; diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/TestMVCCAsyncBytesStoreImpl.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/TestMVCCAsyncBytesStoreImpl.java index 520c5e63ba1..7bcf6657760 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/TestMVCCAsyncBytesStoreImpl.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/TestMVCCAsyncBytesStoreImpl.java @@ -35,7 +35,7 @@ import java.util.concurrent.CompletableFuture; import java.util.stream.IntStream; import lombok.Cleanup; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.api.kv.op.PutOp; import org.apache.bookkeeper.api.kv.options.Options; import org.apache.bookkeeper.api.kv.result.Code; @@ -60,7 +60,7 @@ /** * Unit test of {@link MVCCAsyncBytesStoreImpl}. */ -@Slf4j +@CustomLog public class TestMVCCAsyncBytesStoreImpl extends TestDistributedLogBase { @Rule @@ -438,14 +438,14 @@ public void testReplayJournal() throws Exception { try { result(store.putIfAbsent(getKey(i), getValue(100 + i))); } catch (Exception e) { - log.error("Failed to put kv pair ({})", i, e); + log.error().attr("i", i).exception(e).log("Failed to put kv pair"); } }); - log.info("Closing the store '{}' ...", streamName); + log.info().attr("streamName", streamName).log("Closing the store ..."); // close the store store.close(); - log.info("Closed the store '{}' ...", streamName); + log.info().attr("streamName", streamName).log("Closed the store ..."); // open the store again to replay the journal. store = new MVCCAsyncBytesStoreImpl( diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/TestMVCCStoreImpl.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/TestMVCCStoreImpl.java index adb023412b5..5fc023ec042 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/TestMVCCStoreImpl.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/TestMVCCStoreImpl.java @@ -29,7 +29,7 @@ import java.io.IOException; import java.util.List; import lombok.Cleanup; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.api.kv.op.CompareResult; import org.apache.bookkeeper.api.kv.op.OpType; import org.apache.bookkeeper.api.kv.op.RangeOp; @@ -57,7 +57,7 @@ /** * Unit test of {@link MVCCStoreImpl}. */ -@Slf4j +@CustomLog public class TestMVCCStoreImpl { @Rule diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointerTest.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointerTest.java index 9fe2f116409..e14b38b4781 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointerTest.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointerTest.java @@ -43,7 +43,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import lombok.Cleanup; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.coder.StringUtf8Coder; import org.apache.bookkeeper.common.kv.KV; import org.apache.bookkeeper.statelib.api.StateStoreSpec; @@ -67,7 +67,7 @@ /** * Unit test of {@link RocksCheckpointer}. */ -@Slf4j +@CustomLog public class RocksCheckpointerTest { @Rule diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/testing/executors/MockExecutorController.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/testing/executors/MockExecutorController.java index a191f3fd104..e50a7e27d0c 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/testing/executors/MockExecutorController.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/testing/executors/MockExecutorController.java @@ -40,9 +40,9 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import lombok.CustomLog; import lombok.Data; import lombok.Getter; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.hamcrest.MatcherAssert; import org.mockito.stubbing.Answer; @@ -51,7 +51,7 @@ * A mocked scheduled executor that records scheduled tasks and executes them when the clock is * advanced past their execution time. */ -@Slf4j +@CustomLog public class MockExecutorController { public static final String THREAD_NAME_PREFIX = "realWriteExecutor-"; @@ -139,7 +139,7 @@ private void runTask(Runnable runnable) { } catch (AssertionError ae) { throw ae; } catch (Throwable t) { - log.error("Got unexpected exception while submitting a Runnable", t); + log.error().exception(t).log("Got unexpected exception while submitting a Runnable"); fail("Got unexpected exception while submitting a Runnable " + t.getMessage()); } } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImpl.java index 80c56f3a92d..c7fffa3d2c9 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImpl.java @@ -23,8 +23,8 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import lombok.AccessLevel; +import lombok.CustomLog; import lombok.Getter; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.discover.RegistrationClient; import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener; import org.apache.bookkeeper.net.BookieId; @@ -38,7 +38,7 @@ /** * A default implementation of {@link ClusterControllerLeader}. */ -@Slf4j +@CustomLog public class ClusterControllerLeaderImpl implements ClusterControllerLeader, RegistrationListener { // the metadata store for reading and writing cluster metadata. @@ -117,7 +117,8 @@ public void processAsLeader() throws Exception { try { this.regClient.watchWritableBookies(this).get(); } catch (Exception e) { - log.warn("Controller leader fails to watch servers : {}, giving up leadership", e.getMessage()); + log.warn().attr("exceptionMessage", e.getMessage()) + .log("Controller leader fails to watch servers, giving up leadership"); throw e; } @@ -165,7 +166,8 @@ private void processServerChange() throws InterruptedException { long elapsedMs = System.currentTimeMillis() - lastSuccessfulAssignmentAt; long remainingMs = scheduleDuration.toMillis() - elapsedMs; if (remainingMs > 0) { - log.info("Waiting {} milliseconds for controller to assign containers", remainingMs); + log.info().attr("remainingMs", remainingMs) + .log("Waiting for controller to assign containers"); TimeUnit.MILLISECONDS.sleep(remainingMs); } @@ -199,9 +201,7 @@ private void processServerChange() throws InterruptedException { if (newState.equals(currentState)) { // no assignment state is changed, so do nothing - if (log.isDebugEnabled()) { - log.debug("Assignment state is unchanged - {}", newState); - } + log.debug().attr("newState", newState).log("Assignment state is unchanged"); } else { // update the assignment state lastSuccessfulAssignmentAt = System.currentTimeMillis(); @@ -211,7 +211,7 @@ private void processServerChange() throws InterruptedException { @Override public void onBookiesChanged(Versioned> bookies) { - log.info("Cluster topology is changed - new cluster : {}", bookies); + log.info().attr("bookies", bookies).log("Cluster topology is changed"); // when bookies are changed, notify the leader to take actions this.availableServers = bookies.getValue(); performServerChangesPermits.release(); diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterControllerLeaderSelector.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterControllerLeaderSelector.java index 4f4a096a8ac..38b955558cb 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterControllerLeaderSelector.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterControllerLeaderSelector.java @@ -21,7 +21,7 @@ import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.bookkeeper.stream.storage.StorageConstants.getControllerPath; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.stream.storage.api.cluster.ClusterControllerLeader; import org.apache.bookkeeper.stream.storage.api.cluster.ClusterControllerLeaderSelector; import org.apache.curator.framework.CuratorFramework; @@ -32,7 +32,7 @@ /** * A controller leader selector implemented using zookeeper. */ -@Slf4j +@CustomLog public class ZkClusterControllerLeaderSelector implements ClusterControllerLeaderSelector, ConnectionStateListener { private final CuratorFramework client; diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterControllerLeaderSelectorListener.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterControllerLeaderSelectorListener.java index 87af0f403d2..077fed36f5f 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterControllerLeaderSelectorListener.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterControllerLeaderSelectorListener.java @@ -18,7 +18,7 @@ */ package org.apache.bookkeeper.stream.storage.impl.cluster; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.stream.storage.api.cluster.ClusterControllerLeader; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.leader.LeaderSelectorListener; @@ -30,7 +30,7 @@ *

    The controller leader is responsible for making sure all storage containers are assigned to servers, and load * balancing storage containers if necessary. */ -@Slf4j +@CustomLog class ZkClusterControllerLeaderSelectorListener implements LeaderSelectorListener { private final ClusterControllerLeader controller; @@ -47,7 +47,7 @@ public void takeLeadership(CuratorFramework curatorFramework) throws Exception { @Override public void stateChanged(CuratorFramework client, ConnectionState newState) { - log.info("zookeeper connection state changed to {} for cluster controller", newState); + log.info().attr("newState", newState).log("zookeeper connection state changed for cluster controller"); } } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java index f50ed0b50a3..ecc1874608c 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterInitializer.java @@ -24,7 +24,7 @@ import com.google.common.base.Strings; import java.net.URI; import java.util.Optional; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.stream.proto.cluster.ClusterMetadata; import org.apache.bookkeeper.stream.storage.api.cluster.ClusterInitializer; @@ -37,7 +37,7 @@ /** * ZooKeeper Based Cluster Initializer. */ -@Slf4j +@CustomLog public class ZkClusterInitializer implements ClusterInitializer { private final String zkExternalConnectString; @@ -70,7 +70,7 @@ public boolean initializeCluster(URI metadataServiceUri, int numStorageContainer ClusterMetadata metadata; try { metadata = store.getClusterMetadata(); - log.info("Loaded cluster metadata : \n{}", metadata); + log.info().attr("metadata", metadata).log("Loaded cluster metadata"); return false; } catch (StorageRuntimeException sre) { if (sre.getCause() instanceof KeeperException.NoNodeException) { @@ -83,11 +83,15 @@ public boolean initializeCluster(URI metadataServiceUri, int numStorageContainer segmentStorePath = Optional.of(ledgersPath); } - log.info("Initializing the stream cluster with {} storage containers with segment store path {}.", - numStorageContainers, segmentStorePath.orElse(getSegmentsRootPath(ZK_METADATA_ROOT_PATH))); + log.info() + .attr("numStorageContainers", numStorageContainers) + .attr("segmentStorePath", + segmentStorePath.orElse(getSegmentsRootPath(ZK_METADATA_ROOT_PATH))) + .log("Initializing the stream cluster"); boolean initialized = store.initializeCluster(numStorageContainers, segmentStorePath); - log.info("Successfully initialized the stream cluster : \n{}", store.getClusterMetadata()); + log.info().attr("metadata", store.getClusterMetadata()) + .log("Successfully initialized the stream cluster"); return initialized; } else { throw sre; diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterMetadataStore.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterMetadataStore.java index 0d2fd45c47d..e583fc29597 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterMetadataStore.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterMetadataStore.java @@ -32,7 +32,7 @@ import java.util.Optional; import java.util.concurrent.Executor; import java.util.function.Consumer; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.stream.proto.cluster.ClusterAssignmentData; import org.apache.bookkeeper.stream.proto.cluster.ClusterMetadata; import org.apache.bookkeeper.stream.storage.api.cluster.ClusterMetadataStore; @@ -46,7 +46,7 @@ /** * A zookeeper based implementation of cluster metadata store. */ -@Slf4j +@CustomLog @SuppressWarnings("deprecation") public class ZkClusterMetadataStore implements ClusterMetadataStore { @@ -87,7 +87,7 @@ public void close() { try { assignmentDataCache.close(); } catch (IOException e) { - log.warn("Failed to close assignment data cache", e); + log.warn().exception(e).log("Failed to close assignment data cache"); } } } @@ -183,7 +183,7 @@ public void unwatchClusterAssignmentData(Consumer watcher) { try { assignmentDataCache.close(); } catch (IOException e) { - log.warn("Failed to close assignment data cache when there is no watcher", e); + log.warn().exception(e).log("Failed to close assignment data cache when there is no watcher"); } } } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcMetaRangeService.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcMetaRangeService.java index 1fb0e448a2c..ff269b3d5fc 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcMetaRangeService.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcMetaRangeService.java @@ -18,7 +18,7 @@ package org.apache.bookkeeper.stream.storage.impl.grpc; import io.grpc.stub.StreamObserver; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.stream.proto.storage.GetActiveRangesRequest; import org.apache.bookkeeper.stream.proto.storage.GetActiveRangesResponse; import org.apache.bookkeeper.stream.proto.storage.MetaRangeServiceGrpc.MetaRangeServiceImplBase; @@ -29,7 +29,7 @@ /** * The gRPC protocol based range service. */ -@Slf4j +@CustomLog public class GrpcMetaRangeService extends MetaRangeServiceImplBase { private final RangeStoreService rangeStore; diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcTableService.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcTableService.java index f26a7bd962c..d7fdfdb4b02 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcTableService.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcTableService.java @@ -18,7 +18,7 @@ package org.apache.bookkeeper.stream.storage.impl.grpc; import io.grpc.stub.StreamObserver; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.stream.proto.kv.rpc.DeleteRangeRequest; import org.apache.bookkeeper.stream.proto.kv.rpc.DeleteRangeResponse; import org.apache.bookkeeper.stream.proto.kv.rpc.IncrementRequest; @@ -38,7 +38,7 @@ /** * The gRPC protocol based k/v service. */ -@Slf4j +@CustomLog public class GrpcTableService extends TableServiceImplBase { private final RangeStoreService rangeStore; diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImpl.java index 9a9670ae0d5..13379656c76 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImpl.java @@ -28,7 +28,7 @@ import com.google.protobuf.ByteString; import java.util.concurrent.CompletableFuture; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.api.kv.op.DeleteOp; import org.apache.bookkeeper.api.kv.op.IncrementOp; import org.apache.bookkeeper.api.kv.op.Op; @@ -62,7 +62,7 @@ /** * A table store implementation based on {@link org.apache.bookkeeper.statelib.api.mvcc.MVCCAsyncStore}. */ -@Slf4j +@CustomLog public class TableStoreImpl implements TableStore { private final MVCCAsyncStore store; @@ -73,9 +73,7 @@ public TableStoreImpl(MVCCAsyncStore store) { @Override public CompletableFuture range(RangeRequest rangeReq) { - if (log.isTraceEnabled()) { - log.trace("Received range request {}", rangeReq); - } + log.trace().attr("request", rangeReq).log("Received range request"); return doRange(rangeReq) .thenApply(result -> { try { @@ -88,7 +86,7 @@ public CompletableFuture range(RangeRequest rangeReq) { } }) .exceptionally(cause -> { - log.error("Failed to process range request {}", rangeReq, cause); + log.error().attr("request", rangeReq).exception(cause).log("Failed to process range request"); return RangeResponse.newBuilder() .setHeader(ResponseHeader.newBuilder() .setCode(handleCause(cause)) @@ -152,7 +150,7 @@ public CompletableFuture put(PutRequest putReq) { } }) .exceptionally(cause -> { - log.error("Failed to process put request {}", putReq, cause); + log.error().attr("request", putReq).exception(cause).log("Failed to process put request"); return PutResponse.newBuilder() .setHeader(ResponseHeader.newBuilder() .setCode(handleCause(cause)) @@ -193,7 +191,7 @@ public CompletableFuture incr(IncrementRequest incrementReq) } }) .exceptionally(cause -> { - log.error("Failed to process increment request {}", incrementReq, cause); + log.error().attr("request", incrementReq).exception(cause).log("Failed to process increment request"); return IncrementResponse.newBuilder() .setHeader(ResponseHeader.newBuilder() .setCode(handleCause(cause)) @@ -266,9 +264,7 @@ private DeleteOp buildDeleteOp(RoutingHeader header, DeleteRange @Override public CompletableFuture txn(TxnRequest txnReq) { - if (log.isTraceEnabled()) { - log.trace("Received txn request : {}", txnReq); - } + log.trace().attr("request", txnReq).log("Received txn request"); return doTxn(txnReq) .thenApply(txnResult -> { try { diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreUtils.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreUtils.java index 964ca621ad8..c7eb2a42e8b 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreUtils.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreUtils.java @@ -25,8 +25,8 @@ import io.netty.buffer.Unpooled; import java.util.concurrent.ExecutionException; import lombok.AccessLevel; +import lombok.CustomLog; import lombok.NoArgsConstructor; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.api.kv.op.CompareOp; import org.apache.bookkeeper.api.kv.op.CompareResult; import org.apache.bookkeeper.api.kv.op.OpFactory; @@ -53,7 +53,7 @@ /** * Utils for accessing table stores. */ -@Slf4j +@CustomLog @NoArgsConstructor(access = AccessLevel.PRIVATE) final class TableStoreUtils { diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImpl.java index b64d3205f71..117da800657 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImpl.java @@ -22,7 +22,7 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.impl.internal.api.StorageServerClientManager; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.statelib.api.mvcc.MVCCAsyncStore; @@ -41,7 +41,7 @@ /** * The default implementation of {@link MetaRangeStore}. */ -@Slf4j +@CustomLog public class MetaRangeStoreImpl implements MetaRangeStore { diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java index 507cb56c701..284258f70cf 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java @@ -26,7 +26,7 @@ import com.google.protobuf.InvalidProtocolBufferException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.api.kv.op.CompareResult; import org.apache.bookkeeper.api.kv.op.RangeOp; import org.apache.bookkeeper.api.kv.op.TxnOp; @@ -60,7 +60,7 @@ /** * A statestore backed implementation of {@link RootRangeStore}. */ -@Slf4j +@CustomLog public class RootRangeStoreImpl implements RootRangeStore { @@ -179,9 +179,7 @@ CompletableFuture> getValue(byte[] key) { @Override public CompletableFuture createNamespace(CreateNamespaceRequest request) { - if (log.isTraceEnabled()) { - log.trace("Received CreateNamespace request : {}", request); - } + log.trace().attr("request", request).log("Received CreateNamespace request"); return CreateNamespaceProcessor.of().process( this, @@ -193,7 +191,7 @@ StatusCode verifyCreateNamespaceRequest(CreateNamespaceRequest request) { String colName = request.getName(); StatusCode code = StatusCode.SUCCESS; if (!validateNamespaceName(colName)) { - log.error("Failed to create namespace due to invalid namespace name {}", colName); + log.error().attr("nsName", colName).log("Failed to create namespace due to invalid namespace name"); code = StatusCode.INVALID_NAMESPACE_NAME; } return code; @@ -278,7 +276,7 @@ StatusCode verifyDeleteNamespaceRequest(DeleteNamespaceRequest request) { String colName = request.getName(); StatusCode code = StatusCode.SUCCESS; if (!validateNamespaceName(colName)) { - log.error("Failed to delete namespace due to invalid namespace name {}", colName); + log.error().attr("nsName", colName).log("Failed to delete namespace due to invalid namespace name"); code = StatusCode.INVALID_NAMESPACE_NAME; } return code; @@ -336,7 +334,7 @@ CompletableFuture deleteNamespace(String nsName, Namesp public CompletableFuture getNamespace(GetNamespaceRequest request) { String nsName = request.getName(); if (!validateNamespaceName(nsName)) { - log.error("Failed to get namespace due to invalid namespace name {}", nsName); + log.error().attr("nsName", nsName).log("Failed to get namespace due to invalid namespace name"); return FutureUtils.value( GetNamespaceResponse.newBuilder() .setCode(StatusCode.INVALID_NAMESPACE_NAME) @@ -399,10 +397,10 @@ public CompletableFuture createStream(CreateStreamRequest StatusCode verifyStreamRequest(String nsName, String streamName) { StatusCode code = StatusCode.SUCCESS; if (!validateNamespaceName(nsName)) { - log.error("Invalid namespace name {}", nsName); + log.error().attr("nsName", nsName).log("Invalid namespace name"); code = StatusCode.INVALID_NAMESPACE_NAME; } else if (!validateStreamName(streamName)) { - log.error("Invalid stream name {}", streamName); + log.error().attr("streamName", streamName).log("Invalid stream name"); code = StatusCode.INVALID_STREAM_NAME; } return code; @@ -450,8 +448,13 @@ private CompletableFuture createStream(NamespaceMetadata n } private static void logStreamOp(String op, long nsId, long streamId, String streamName, StatusCode result) { - log.info("stream {} namespace_id={} stream_id={} stream_name={} result={}", - op, nsId, streamId, streamName, result); + log.info() + .attr("op", op) + .attr("namespaceId", nsId) + .attr("streamId", streamId) + .attr("streamName", streamName) + .attr("result", result) + .log("stream op"); } private CompletableFuture executeCreateStreamTxn(long nsId, diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/MetaRangeImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/MetaRangeImpl.java index 6792802ad64..e1c12537e09 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/MetaRangeImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/MetaRangeImpl.java @@ -36,7 +36,7 @@ import java.util.function.Consumer; import java.util.function.Supplier; import javax.annotation.concurrent.GuardedBy; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.api.kv.op.CompareResult; import org.apache.bookkeeper.api.kv.op.Op; import org.apache.bookkeeper.api.kv.op.TxnOp; @@ -60,7 +60,7 @@ /** * The default implementation of {@link MetaRange}. */ -@Slf4j +@CustomLog public class MetaRangeImpl implements MetaRange { private static final byte METADATA_SEP = (byte) 0x1; @@ -324,15 +324,11 @@ private void unsafeCreate(CompletableFuture createFuture, .Then(successOps.toArray(new Op[successOps.size()])) .build(); - if (log.isTraceEnabled()) { - log.trace("Execute create stream metadata range txn {}", streamProps); - } + log.trace().attr("streamProps", streamProps).log("Execute create stream metadata range txn"); store.txn(txn) .thenApplyAsync(txnResult -> { try { - if (log.isTraceEnabled()) { - log.trace("Create stream metadata range txn result = {}", txnResult.isSuccess()); - } + log.trace().attr("success", txnResult.isSuccess()).log("Create stream metadata range txn result"); if (txnResult.isSuccess()) { List> results = txnResult.results(); MetaRangeImpl.this.revision = results.get(results.size() - 1).revision(); diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/routing/RangeRoutingTableImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/routing/RangeRoutingTableImpl.java index 13fd5478f2b..75b715dbb5a 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/routing/RangeRoutingTableImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/routing/RangeRoutingTableImpl.java @@ -21,7 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import java.util.concurrent.CompletableFuture; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.impl.internal.api.StorageServerClientManager; import org.apache.bookkeeper.clients.impl.routing.RangeRouter; import org.apache.bookkeeper.common.concurrent.FutureUtils; @@ -32,7 +32,7 @@ /** * A default implementation of {@link RangeRoutingTable}. */ -@Slf4j +@CustomLog public class RangeRoutingTableImpl implements RangeRoutingTable { private final StorageServerClientManager manager; diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptor.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptor.java index ce529c5dd5e..34cad9967c8 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptor.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptor.java @@ -43,8 +43,8 @@ import java.io.InputStream; import java.util.HashMap; import java.util.Map; +import lombok.CustomLog; import lombok.Data; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.stream.proto.kv.rpc.DeleteRangeRequest; import org.apache.bookkeeper.stream.proto.kv.rpc.IncrementRequest; import org.apache.bookkeeper.stream.proto.kv.rpc.PutRequest; @@ -57,7 +57,7 @@ /** * A client interceptor that intercepting kv rpcs to attach routing information. */ -@Slf4j +@CustomLog public class RoutingHeaderProxyInterceptor implements ClientInterceptor { /** @@ -164,12 +164,11 @@ private static class InterceptorDescriptor { public ClientCall interceptCall(MethodDescriptor method, CallOptions callOptions, Channel next) { - if (log.isTraceEnabled()) { - log.trace("Intercepting method {} : req marshaller = {}, resp marshaller = {}", - method.getFullMethodName(), - method.getRequestMarshaller(), - method.getResponseMarshaller()); - } + log.trace() + .attr("method", method.getFullMethodName()) + .attr("requestMarshaller", method.getRequestMarshaller()) + .attr("responseMarshaller", method.getResponseMarshaller()) + .log("Intercepting method"); InterceptorDescriptor descriptor = kvRpcMethods.get(method.getFullMethodName()); return new SimpleForwardingClientCall(next.newCall(method, callOptions)) { @@ -183,10 +182,11 @@ public void start(Listener responseListener, Metadata headers) { sid = headers.get(SID_METADATA_KEY); rid = headers.get(RID_METADATA_KEY); rk = headers.get(RK_METADATA_KEY); - if (log.isTraceEnabled()) { - log.trace("Intercepting request with header : sid = {}, rid = {}, rk = {}", - sid, rid, rk); - } + log.trace() + .attr("streamId", sid) + .attr("rangeId", rid) + .attr("routingKey", rk) + .log("Intercepting request with header"); delegate().start(responseListener, headers); } @@ -220,14 +220,14 @@ private ReqT interceptMessage(MethodDescriptor method try { bytes = is.available(); } catch (IOException e) { - log.warn("Encountered exceptions in getting available bytes of message", e); + log.warn().exception(e).log("Encountered exceptions in getting available bytes of message"); throw new RuntimeException("Encountered exception in intercepting message", e); } ByteBuf buffer = PooledByteBufAllocator.DEFAULT.buffer(); try { buffer.writeBytes(is, bytes); } catch (IOException e) { - log.warn("Encountered exceptions in transferring bytes to the buffer", e); + log.warn().exception(e).log("Encountered exceptions in transferring bytes to the buffer"); ReferenceCountUtil.release(buffer); throw new RuntimeException("Encountered exceptions in transferring bytes to the buffer", e); } @@ -250,8 +250,12 @@ private ReqT interceptMessage( try { return interceptTableRequest(method, descriptor, message, sid, rid, rk); } catch (Throwable t) { - log.error("Failed to intercept table request (sid = {}, rid = {}, rk = {}) : ", - sid, rid, Hex.encodeHexString(rk), t); + log.error() + .attr("streamId", sid) + .attr("rangeId", rid) + .attr("routingKey", Hex.encodeHexString(rk)) + .exception(t) + .log("Failed to intercept table request"); return message; } } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerController.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerController.java index 271587d4294..ecafa8fcf46 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerController.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerController.java @@ -31,7 +31,7 @@ import java.util.TreeSet; import java.util.stream.Collectors; import java.util.stream.LongStream; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stream.proto.cluster.ClusterAssignmentData; import org.apache.bookkeeper.stream.proto.cluster.ClusterMetadata; @@ -45,7 +45,7 @@ * *

    The algorithm here is based on the count-based stream balancer in distributedlog-proxy-server. */ -@Slf4j +@CustomLog public class DefaultStorageContainerController implements StorageContainerController { static final class ServerAssignmentDataComparator @@ -90,8 +90,10 @@ public ClusterAssignmentData computeIdealState(ClusterMetadata clusterMetadata, e2 -> e2.getValue().getContainersList().stream().collect(Collectors.toSet()) )); } catch (UncheckedExecutionException uee) { - log.warn("Invalid cluster assignment data is found : {} - {}. Recompute assignment from empty state", - currentState, uee.getCause().getMessage()); + log.warn() + .attr("currentState", currentState) + .attr("cause", uee.getCause().getMessage()) + .log("Invalid cluster assignment data is found. Recompute assignment from empty state"); currentServerAssignments = Maps.newHashMap(); } Set currentServersAssigned = currentServerAssignments.keySet(); @@ -112,9 +114,12 @@ public ClusterAssignmentData computeIdealState(ClusterMetadata clusterMetadata, return currentState; } - log.info("Storage container controller detects cluster changed:\n" - + "\t {} servers added: {}\n\t {} servers removed: {}", - serversAdded.size(), serversAdded, serversRemoved.size(), serversRemoved); + log.info() + .attr("numAdded", serversAdded.size()) + .attr("serversAdded", serversAdded) + .attr("numRemoved", serversRemoved.size()) + .attr("serversRemoved", serversRemoved) + .log("Storage container controller detects cluster changed"); // 4. compute the containers that owned by servers removed. these containers are needed to be reassigned. Set containersToReassign = currentServerAssignments.entrySet().stream() @@ -129,14 +134,10 @@ public ClusterAssignmentData computeIdealState(ClusterMetadata clusterMetadata, BookieId host = entry.getKey(); if (!currentCluster.contains(host)) { - if (log.isTraceEnabled()) { - log.trace("Host {} is not in current cluster anymore", host); - } + log.trace().attr("host", host).log("Host is not in current cluster anymore"); continue; } else { - if (log.isTraceEnabled()) { - log.trace("Adding host {} to assignment queue", host); - } + log.trace().attr("host", host).log("Adding host to assignment queue"); assignmentQueue.add(Pair.of(host, Lists.newLinkedList(entry.getValue()))); } } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerImpl.java index b3c67f20fde..68d3009f6b1 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerImpl.java @@ -27,7 +27,7 @@ import java.io.IOException; import java.util.Collection; import java.util.concurrent.CompletableFuture; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.stream.storage.api.sc.StorageContainer; import org.apache.bookkeeper.stream.storage.api.sc.StorageContainerService; @@ -37,7 +37,7 @@ /** * The default implementation of {@link StorageContainer}. */ -@Slf4j +@CustomLog class StorageContainerImpl implements StorageContainer { private final String containerName; @@ -73,12 +73,12 @@ public long getId() { @Override public CompletableFuture start() { - log.info("Starting storage container ({}) ...", getId()); + log.info().attr("scId", getId()).log("Starting storage container"); return service.start().thenCompose(ignored -> { try { grpcServer.start(); - log.info("Successfully started storage container ({}).", getId()); + log.info().attr("scId", getId()).log("Successfully started storage container"); channel = InProcessChannelBuilder.forName(containerName) .usePlaintext() @@ -88,7 +88,8 @@ public CompletableFuture start() { .build(); return FutureUtils.value(StorageContainerImpl.this); } catch (IOException e) { - log.error("Failed to start the grpc server for storage container ({})", getId(), e); + log.error().attr("scId", getId()).exception(e) + .log("Failed to start the grpc server for storage container"); return FutureUtils.exception(e); } }); @@ -96,7 +97,7 @@ public CompletableFuture start() { @Override public CompletableFuture stop() { - log.info("Stopping storage container ({}) ...", getId()); + log.info().attr("scId", getId()).log("Stopping storage container"); Channel existingChannel = channel; @@ -115,7 +116,7 @@ public CompletableFuture stop() { return service.stop().thenApply(ignored -> { - log.info("Successfully stopped storage container ({}).", getId()); + log.info().attr("scId", getId()).log("Successfully stopped storage container"); return null; }); } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerRegistryImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerRegistryImpl.java index 4208b1a0644..1c5913f7909 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerRegistryImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/StorageContainerRegistryImpl.java @@ -19,7 +19,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.locks.ReentrantReadWriteLock; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.exceptions.ObjectClosedException; import org.apache.bookkeeper.stream.storage.api.sc.StorageContainer; @@ -30,7 +30,7 @@ /** * The default implementation of {@link StorageContainerRegistry}. */ -@Slf4j +@CustomLog public class StorageContainerRegistryImpl implements StorageContainerRegistry { private static final String COMPONENT_NAME = StorageContainerRegistry.class.getSimpleName(); @@ -92,22 +92,25 @@ private CompletableFuture unsafeStartStorageContainer(long scI return FutureUtils.exception(new StorageException("StorageContainer " + scId + " already registered")); } - log.info("Registered StorageContainer ('{}').", scId); + log.info().attr("scId", scId).log("Registered StorageContainer"); return newStorageContainer.start() .whenComplete((container, cause) -> { if (null != cause) { if (containers.remove(scId, newStorageContainer)) { - log.warn("De-registered StorageContainer ('{}') when failed to start", scId, cause); + log.warn().attr("scId", scId).exception(cause) + .log("De-registered StorageContainer when failed to start"); } else { - log.warn("Fail to de-register StorageContainer ('{}') when failed to start", scId, cause); + log.warn().attr("scId", scId).exception(cause) + .log("Fail to de-register StorageContainer when failed to start"); } - log.info("Release resources hold by StorageContainer ('{}') during de-register", scId); + log.info().attr("scId", scId) + .log("Release resources hold by StorageContainer during de-register"); newStorageContainer.stop().exceptionally(throwable -> { - log.error("Stop StorageContainer ('{}') fail during de-register", scId); + log.error().attr("scId", scId).log("Stop StorageContainer fail during de-register"); return null; }); } else { - log.info("Successfully started registered StorageContainer ('{}').", scId); + log.info().attr("scId", scId).log("Successfully started registered StorageContainer"); } }); } @@ -130,7 +133,7 @@ private CompletableFuture unsafeStopStorageContainer(long scId, StorageCon if (null == container) { StorageContainer existingContainer = containers.remove(scId); if (null != existingContainer) { - log.info("Unregistered StorageContainer ('{}').", scId); + log.info().attr("scId", scId).log("Unregistered StorageContainer"); return existingContainer.stop(); } else { return FutureUtils.Void(); @@ -139,7 +142,7 @@ private CompletableFuture unsafeStopStorageContainer(long scId, StorageCon boolean removed = containers.remove(scId, container); if (removed) { - log.info("Unregistered StorageContainer ('{}').", scId); + log.info().attr("scId", scId).log("Unregistered StorageContainer"); } // no matter we successfully removed the containers or not, we need to close the current container. diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/ZkStorageContainerManager.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/ZkStorageContainerManager.java index 8684a8ae877..6868103a580 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/ZkStorageContainerManager.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/sc/ZkStorageContainerManager.java @@ -35,8 +35,8 @@ import java.util.function.Consumer; import java.util.stream.Collectors; import lombok.AccessLevel; +import lombok.CustomLog; import lombok.Getter; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.clients.utils.NetUtils; import org.apache.bookkeeper.common.component.AbstractLifecycleComponent; import org.apache.bookkeeper.common.concurrent.FutureUtils; @@ -54,7 +54,7 @@ /** * A zookeeper based implementation of {@link StorageContainerManager}. */ -@Slf4j +@CustomLog public class ZkStorageContainerManager extends AbstractLifecycleComponent implements StorageContainerManager, Consumer { @@ -108,7 +108,8 @@ protected void doStart() { // schedule the container probe task containerProbeTask = executor.scheduleAtFixedRate( this::probeContainers, 0, probeInterval.toMillis(), TimeUnit.MILLISECONDS); - log.info("Scheduled storage container probe task at every {} ms", probeInterval.toMillis()); + log.info().attr("probeIntervalMs", probeInterval.toMillis()) + .log("Scheduled storage container probe task"); } @Override @@ -185,7 +186,7 @@ private boolean refreshMyAssignment() { private void processServersJoined(Set serversJoined, Map newAssignmentMap) { if (!serversJoined.isEmpty()) { - log.info("Servers joined : {}", serversJoined); + log.info().attr("servers", serversJoined).log("Servers joined"); } serversJoined.forEach(ep -> { ServerAssignmentData sad = newAssignmentMap.get(ep); @@ -198,7 +199,7 @@ private void processServersJoined(Set serversJoined, private void processServersLeft(Set serversLeft, Map oldAssignmentMap) { if (!serversLeft.isEmpty()) { - log.info("Servers left : {}", serversLeft); + log.info().attr("servers", serversLeft).log("Servers left"); } serversLeft.forEach(ep -> { ServerAssignmentData sad = oldAssignmentMap.get(ep); @@ -219,8 +220,11 @@ private void processServersAssignmentChanged(Set commonServers, if (oldSad.equals(newSad)) { return; } else { - log.info("Server assignment is change for {}:\nold assignment: {}\nnew assignment: {}", - NetUtils.endpointToString(ep), oldSad, newSad); + log.info() + .attr("server", NetUtils.endpointToString(ep)) + .attr("oldAssignment", oldSad) + .attr("newAssignment", newSad) + .log("Server assignment is changed"); oldSad.getContainersList().forEach(container -> containerAssignmentMap.remove(container, ep)); newSad.getContainersList().forEach(container -> containerAssignmentMap.put(container, ep)); } @@ -251,13 +255,13 @@ private void processMyAssignment(ServerAssignmentData myAssignment) { Sets.filter(containersToStop, container -> !pendingStartStopContainers.contains(container)); if (!containersToStart.isEmpty() || !containersToStop.isEmpty()) { - log.info("Process container changes:\n\tIdeal = {}\n\tLive = {}\n\t" - + "Pending = {}\n\tToStart = {}\n\tToStop = {}", - assignedContainerSet, - liveContainerSet, - pendingStartStopContainers, - containersToStart, - containersToStop); + log.info() + .attr("ideal", assignedContainerSet) + .attr("live", liveContainerSet) + .attr("pending", pendingStartStopContainers) + .attr("toStart", containersToStart) + .attr("toStop", containersToStop) + .log("Process container changes"); } containersToStart.forEach(this::startStorageContainer); @@ -265,10 +269,10 @@ private void processMyAssignment(ServerAssignmentData myAssignment) { } private CompletableFuture startStorageContainer(long scId) { - log.info("Starting storage container ({})", scId); + log.info().attr("scId", scId).log("Starting storage container"); StorageContainer sc = liveContainers.get(scId); if (null != sc) { - log.warn("Storage container ({}) is already started", scId); + log.warn().attr("scId", scId).log("Storage container is already started"); return FutureUtils.value(sc); } else { // register the container to pending list @@ -278,9 +282,10 @@ private CompletableFuture startStorageContainer(long scId) { .whenComplete((container, cause) -> { try { if (null != cause) { - log.warn("Failed to start storage container ({})", scId, cause); + log.warn().attr("scId", scId).exception(cause) + .log("Failed to start storage container"); } else { - log.info("Successfully started storage container ({})", scId); + log.info().attr("scId", scId).log("Successfully started storage container"); addStorageContainer(scId, container); } } finally { @@ -291,10 +296,10 @@ private CompletableFuture startStorageContainer(long scId) { } private CompletableFuture stopStorageContainer(long scId) { - log.info("Stopping storage container ({})", scId); + log.info().attr("scId", scId).log("Stopping storage container"); StorageContainer sc = liveContainers.get(scId); if (null == sc) { - log.warn("Storage container ({}) is not alive anymore", scId); + log.warn().attr("scId", scId).log("Storage container is not alive anymore"); return FutureUtils.Void(); } else { // register the container to pending list @@ -304,9 +309,10 @@ private CompletableFuture stopStorageContainer(long scId) { .whenComplete((container, cause) -> { try { if (cause != null) { - log.warn("Failed to stop storage container ({})", scId, cause); + log.warn().attr("scId", scId).exception(cause) + .log("Failed to stop storage container"); } else { - log.info("Successfully stopped storage container ({})", scId); + log.info().attr("scId", scId).log("Successfully stopped storage container"); removeStorageContainer(scId, sc); } @@ -320,10 +326,10 @@ private CompletableFuture stopStorageContainer(long scId) { private StorageContainer addStorageContainer(long scId, StorageContainer sc) { StorageContainer oldSc = liveContainers.putIfAbsent(scId, sc); if (null == oldSc) { - log.info("Storage container ({}) is added to live set.", sc); + log.info().attr("storageContainer", sc).log("Storage container is added to live set"); return sc; } else { - log.warn("Storage container ({}) has already been added to live set", sc); + log.warn().attr("storageContainer", sc).log("Storage container has already been added to live set"); sc.stop(); return oldSc; } @@ -331,9 +337,9 @@ private StorageContainer addStorageContainer(long scId, StorageContainer sc) { private void removeStorageContainer(long scId, StorageContainer sc) { if (liveContainers.remove(scId, sc)) { - log.info("Storage container ({}) is removed from live set.", scId); + log.info().attr("scId", scId).log("Storage container is removed from live set"); } else { - log.warn("Storage container ({}) can't be removed from live set.", scId); + log.warn().attr("scId", scId).log("Storage container can't be removed from live set"); } } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java index d4f65807445..186b2865469 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java @@ -28,8 +28,8 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import lombok.AccessLevel; +import lombok.CustomLog; import lombok.Getter; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.clients.impl.internal.api.StorageServerClientManager; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.OrderedScheduler; @@ -79,7 +79,7 @@ /** * The service implementation running in a storage container. */ -@Slf4j +@CustomLog class RangeStoreServiceImpl implements RangeStoreService, AutoCloseable { private final long scId; diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/store/MVCCStoreFactoryImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/store/MVCCStoreFactoryImpl.java index 0692fb217f9..0cd058dba50 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/store/MVCCStoreFactoryImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/store/MVCCStoreFactoryImpl.java @@ -29,9 +29,9 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.function.Supplier; import lombok.AccessLevel; +import lombok.CustomLog; import lombok.Getter; import lombok.experimental.Accessors; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.coder.ByteArrayCoder; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.exceptions.ObjectClosedException; @@ -50,7 +50,7 @@ * A default implementation of {@link MVCCStoreFactory}. */ @Accessors(fluent = true) -@Slf4j +@CustomLog public class MVCCStoreFactoryImpl implements MVCCStoreFactory { // store supplier @@ -140,8 +140,11 @@ private synchronized void addStore(long scId, long streamId, long rangeId, if (null != oldStore) { store.closeAsync(); } else { - log.info("Add store (scId = {}, streamId = {}, rangeId = {}) at storage container ({})", - scId, streamId, rangeId, scId); + log.info() + .attr("scId", scId) + .attr("streamId", streamId) + .attr("rangeId", rangeId) + .log("Add store at storage container"); scStores.put(rid, store); } } @@ -174,8 +177,11 @@ CompletableFuture> newStore(long scId, long strea } } - log.info("Initializing stream({})/range({}) at storage container ({})", - streamId, rangeId, scId); + log.info() + .attr("streamId", streamId) + .attr("rangeId", rangeId) + .attr("scId", scId) + .log("Initializing stream/range at storage container"); MVCCAsyncStore store = storeSupplier.get(); @@ -222,17 +228,23 @@ CompletableFuture> newStore(long scId, long strea return store.init(spec).whenComplete((ignored, throwable) -> { // since the store has not been added, so can't release its resources during close sc if (null != throwable) { - log.info("Clearing resources hold by stream({})/range({}) at storage container ({}) ", - streamId, rangeId, scId); + log.info() + .attr("streamId", streamId) + .attr("rangeId", rangeId) + .attr("scId", scId) + .log("Clearing resources hold by stream/range at storage container"); store.closeAsync().whenComplete((i, t) -> { if (null != t) { - log.error("Clear resources hold by {} fail", store.name()); + log.error().attr("store", store.name()).log("Clear resources hold fail"); } }); } }).thenApply(ignored -> { - log.info("Successfully initialize stream({})/range({}) at storage container ({})", - streamId, rangeId, scId); + log.info() + .attr("streamId", streamId) + .attr("rangeId", rangeId) + .attr("scId", scId) + .log("Successfully initialized stream/range at storage container"); addStore(scId, streamId, rangeId, store); return store; }); @@ -245,13 +257,13 @@ public CompletableFuture closeStores(long scId) { scStores = stores.remove(scId); } if (null == scStores) { - log.info("scStores for {} on store factory is null, return directly", scId); + log.info().attr("scId", scId).log("scStores on store factory is null, return directly"); return FutureUtils.Void(); } List> closeFutures = Lists.newArrayList(); for (MVCCAsyncStore store : scStores.values()) { - log.info("Closing {} of sc {}", store.name(), scId); + log.info().attr("store", store.name()).attr("scId", scId).log("Closing store of sc"); closeFutures.add(store.closeAsync()); } diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/TestStorageContainerStoreImpl.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/TestStorageContainerStoreImpl.java index e6d225fa6be..456f0bd3710 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/TestStorageContainerStoreImpl.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/TestStorageContainerStoreImpl.java @@ -47,7 +47,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ThreadLocalRandom; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.impl.container.StorageContainerClientInterceptor; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.grpc.proxy.ProxyHandlerRegistry; @@ -102,7 +102,7 @@ /** * Unit test of {@link StorageContainerStoreImpl}. */ -@Slf4j +@CustomLog public class TestStorageContainerStoreImpl { private static final StreamProperties streamProps = StreamProperties.newBuilder() diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImplTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImplTest.java index a9fbab0fbda..fad0758345e 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImplTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ClusterControllerLeaderImplTest.java @@ -40,7 +40,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.discover.RegistrationClient; import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener; @@ -61,7 +61,7 @@ /** * Unit test {@link ClusterControllerLeaderImpl}. */ -@Slf4j +@CustomLog public class ClusterControllerLeaderImplTest { private static final int NUM_STORAGE_CONTAINERS = 32; @@ -168,7 +168,7 @@ public void testProcessAsLeader() throws Exception { try { clusterController.processAsLeader(); } catch (Exception e) { - log.info("Encountered exception when cluster controller processes as a leader", e); + log.info().exception(e).log("Encountered exception when cluster controller processes as a leader"); } }); diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterControllerLeaderSelectorTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterControllerLeaderSelectorTest.java index c072d004efa..94d736ad569 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterControllerLeaderSelectorTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterControllerLeaderSelectorTest.java @@ -22,7 +22,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.stream.storage.api.cluster.ClusterControllerLeader; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; @@ -38,7 +38,7 @@ /** * Unit test of {@link ZkClusterControllerLeaderSelector}. */ -@Slf4j +@CustomLog public class ZkClusterControllerLeaderSelectorTest extends ZooKeeperClusterTestCase { @Rule @@ -121,7 +121,7 @@ public void processAsLeader() throws Exception { try { TimeUnit.SECONDS.sleep(Long.MAX_VALUE); } catch (InterruptedException ie) { - log.info("Leader is interrupted", ie); + log.info().exception(ie).log("Leader is interrupted"); Thread.currentThread().interrupt(); interruptedLatch.countDown(); } @@ -162,7 +162,7 @@ public void processAsLeader() throws Exception { try { TimeUnit.SECONDS.sleep(Long.MAX_VALUE); } catch (InterruptedException ie) { - log.info("Leader is interrupted", ie); + log.info().exception(ie).log("Leader is interrupted"); Thread.currentThread().interrupt(); } } diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterMetadataStoreTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterMetadataStoreTest.java index b53d24929ba..bcf34331e98 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterMetadataStoreTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/cluster/ZkClusterMetadataStoreTest.java @@ -29,7 +29,7 @@ import java.util.concurrent.Executors; import java.util.function.Consumer; import lombok.Cleanup; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.stream.proto.cluster.ClusterAssignmentData; import org.apache.bookkeeper.stream.proto.cluster.ClusterMetadata; @@ -50,7 +50,7 @@ /** * Unit test {@link ZkClusterMetadataStore}. */ -@Slf4j +@CustomLog public class ZkClusterMetadataStoreTest extends ZooKeeperClusterTestCase { private static final int NUM_STORAGE_CONTAINERS = 1024; diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImplTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImplTest.java index 2c7124f8ef4..0dd25aaf837 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImplTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImplTest.java @@ -28,7 +28,7 @@ import com.google.protobuf.ByteString; import java.util.List; import java.util.concurrent.CompletableFuture; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.stream.proto.kv.KeyValue; import org.apache.bookkeeper.stream.proto.kv.rpc.Compare; @@ -52,7 +52,7 @@ /** * Unit test of {@link TableStoreImpl}. */ -@Slf4j +@CustomLog public class TableStoreImplTest extends MVCCAsyncStoreTestBase { private static final long SC_ID = 123L; diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/TestRootRangeStoreImpl.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/TestRootRangeStoreImpl.java index 9cdbf6424df..ed304fa767c 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/TestRootRangeStoreImpl.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/TestRootRangeStoreImpl.java @@ -34,7 +34,7 @@ import static org.junit.Assert.assertTrue; import java.util.concurrent.CompletableFuture; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.common.util.Bytes; import org.apache.bookkeeper.stream.proto.NamespaceConfiguration; @@ -53,7 +53,7 @@ /** * Unit test for {@link RootRangeStoreImpl}. */ -@Slf4j +@CustomLog public class TestRootRangeStoreImpl extends MVCCAsyncStoreTestBase { private final NamespaceConfiguration namespaceConf = diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/TestMetaRangeImpl.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/TestMetaRangeImpl.java index 62982e14ca3..6cee1aba839 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/TestMetaRangeImpl.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/TestMetaRangeImpl.java @@ -30,7 +30,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.stream.LongStream; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.stream.proto.RangeMetadata; import org.apache.bookkeeper.stream.proto.RangeState; @@ -44,7 +44,7 @@ /** * Unit test of {@link MetaRangeImpl}. */ -@Slf4j +@CustomLog public class TestMetaRangeImpl extends MVCCAsyncStoreTestBase { private StreamProperties streamProps; diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptorTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptorTest.java index 43c060b12ca..0442d6a89f8 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptorTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptorTest.java @@ -37,7 +37,7 @@ import io.grpc.stub.StreamObserver; import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.clients.grpc.GrpcClientTestBase; import org.apache.bookkeeper.clients.impl.channel.StorageServerChannel; import org.apache.bookkeeper.stream.proto.kv.rpc.DeleteRangeRequest; @@ -59,7 +59,7 @@ /** * Unit test {@link RoutingHeaderProxyInterceptor}. */ -@Slf4j +@CustomLog public class RoutingHeaderProxyInterceptorTest extends GrpcClientTestBase { private final long streamId = 1234L; @@ -74,7 +74,7 @@ protected void doSetup() { @Override public void range(RangeRequest request, StreamObserver responseObserver) { - log.info("Received range request : {}", request); + log.info().attr("request", request).log("Received range request"); receivedRequest.set(request); responseObserver.onNext(RangeResponse.newBuilder() .setHeader(ResponseHeader.newBuilder() @@ -87,7 +87,7 @@ public void range(RangeRequest request, StreamObserver responseOb @Override public void delete(DeleteRangeRequest request, StreamObserver responseObserver) { - log.info("Received delete range request : {}", request); + log.info().attr("request", request).log("Received delete range request"); receivedRequest.set(request); responseObserver.onNext(DeleteRangeResponse.newBuilder() .setHeader(ResponseHeader.newBuilder() @@ -100,7 +100,7 @@ public void delete(DeleteRangeRequest request, StreamObserver responseObserver) { - log.info("Received txn request : {}", request); + log.info().attr("request", request).log("Received txn request"); receivedRequest.set(request); responseObserver.onNext(TxnResponse.newBuilder() .setHeader(ResponseHeader.newBuilder() @@ -113,7 +113,7 @@ public void txn(TxnRequest request, StreamObserver responseObserver @Override public void increment(IncrementRequest request, StreamObserver responseObserver) { - log.info("Received incr request : {}", request); + log.info().attr("request", request).log("Received incr request"); receivedRequest.set(request); responseObserver.onNext(IncrementResponse.newBuilder() .setHeader(ResponseHeader.newBuilder() @@ -126,7 +126,7 @@ public void increment(IncrementRequest request, StreamObserver responseObserver) { - log.info("Received put request : {}", request); + log.info().attr("request", request).log("Received put request"); receivedRequest.set(request); responseObserver.onNext(PutResponse.newBuilder() .setHeader(ResponseHeader.newBuilder() @@ -153,8 +153,11 @@ public ClientCall interceptCall(MethodDescriptor(next.newCall(method, callOptions)) { @Override protected void checkedStart(Listener responseListener, Metadata headers) { - log.info("Intercept the request with routing information : sid = {}, rid = {}, rk = {}", - streamId, rangeId, new String(routingKey, UTF_8)); + log.info() + .attr("streamId", streamId) + .attr("rangeId", rangeId) + .attr("routingKey", new String(routingKey, UTF_8)) + .log("Intercept the request with routing information"); headers.put( RID_METADATA_KEY, rangeId diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerControllerTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerControllerTest.java index 8d01068a552..f6befb0487e 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerControllerTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/sc/DefaultStorageContainerControllerTest.java @@ -33,7 +33,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.LongStream; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.stream.proto.cluster.ClusterAssignmentData; @@ -46,7 +46,7 @@ /** * Unit test {@link DefaultStorageContainerController}. */ -@Slf4j +@CustomLog public class DefaultStorageContainerControllerTest { private static final int NUM_STORAGE_CONTAINERS = 32; @@ -130,7 +130,8 @@ private static void verifyAssignmentData(ClusterAssignmentData newAssignment, int numContainersPerServer = NUM_STORAGE_CONTAINERS / numServers; int serverIdx = 0; for (Map.Entry entry : newAssignment.getServersMap().entrySet()) { - log.info("Check assignment for server {} = {}", entry.getKey(), entry.getValue()); + log.info().attr("server", entry.getKey()).attr("assignment", entry.getValue()) + .log("Check assignment for server"); BookieId address = BookieId.parse(entry.getKey()); assignedServers.add(address); @@ -173,7 +174,8 @@ private static void verifyAssignmentDataWhenHasMoreServers(ClusterAssignmentData int numAssignedServers = 0; int serverIdx = 0; for (Map.Entry entry : newAssignment.getServersMap().entrySet()) { - log.info("Check assignment for server {} = {}", entry.getKey(), entry.getValue()); + log.info().attr("server", entry.getKey()).attr("assignment", entry.getValue()) + .log("Check assignment for server"); BookieId address = BookieId.parse(entry.getKey()); assignedServers.add(address); diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/store/MVCCStoreFactoryImplTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/store/MVCCStoreFactoryImplTest.java index 577bed280bf..e4e51a34cf6 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/store/MVCCStoreFactoryImplTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/store/MVCCStoreFactoryImplTest.java @@ -32,7 +32,7 @@ import java.io.IOException; import java.nio.file.Paths; import java.time.Duration; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.apache.bookkeeper.common.coder.ByteArrayCoder; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.statelib.api.mvcc.MVCCAsyncStore; @@ -56,7 +56,7 @@ /** * Unit test of {@link MVCCStoreFactoryImpl}. */ -@Slf4j +@CustomLog public class MVCCStoreFactoryImplTest { @Rule @@ -122,8 +122,11 @@ public void testOpenStore() throws Exception { try (MVCCAsyncStore store = FutureUtils.result( factory.openStore(scId, streamId, rangeId, 0))) { - log.info("Open store (scId = {}, streamId = {}, rangeId = {}) to test", - scId, streamId, rangeId); + log.info() + .attr("scId", scId) + .attr("streamId", streamId) + .attr("rangeId", rangeId) + .log("Open store to test"); String storeName = String.format( "%s/%s/%s", diff --git a/stream/tests-common/src/main/java/org/apache/bookkeeper/tests/rpc/PingPongService.java b/stream/tests-common/src/main/java/org/apache/bookkeeper/tests/rpc/PingPongService.java index 4a947762da7..1b6d7af448d 100644 --- a/stream/tests-common/src/main/java/org/apache/bookkeeper/tests/rpc/PingPongService.java +++ b/stream/tests-common/src/main/java/org/apache/bookkeeper/tests/rpc/PingPongService.java @@ -19,7 +19,7 @@ package org.apache.bookkeeper.tests.rpc; import io.grpc.stub.StreamObserver; -import lombok.extern.slf4j.Slf4j; +import lombok.CustomLog; import org.bookkeeper.tests.proto.rpc.PingPongServiceGrpc.PingPongServiceImplBase; import org.bookkeeper.tests.proto.rpc.PingRequest; import org.bookkeeper.tests.proto.rpc.PongResponse; @@ -27,7 +27,7 @@ /** * An implementation of the ping pong service used for testing. */ -@Slf4j +@CustomLog public class PingPongService extends PingPongServiceImplBase { private final int streamPongSize; @@ -61,7 +61,7 @@ public void onNext(PingRequest value) { @Override public void onError(Throwable t) { - log.error("Failed on receiving stream of pings", t); + log.error().exception(t).log("Failed on receiving stream of pings"); } @Override From caff07074d5135ff884609955932eeae3834426e Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 24 Apr 2026 21:16:59 -0700 Subject: [PATCH 3/3] Bump slog dependency to 0.9.8 Pulls in the MDC propagation fix from merlimat/slog#6, which makes log4j2 ThreadContext entries visible on slog events emitted via the Log4j2Logger backend (so %X{key} layouts and appenders that read event.getContextData().getValue(key) see the caller's MDC). --- bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt | 4 ++-- bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt | 4 ++-- bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt | 4 ++-- pom.xml | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt index 868168e9335..f66fb0c91a7 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt @@ -216,7 +216,7 @@ Apache Software License, Version 2. - lib/commons-codec-commons-codec-1.18.0.jar [6] - lib/commons-io-commons-io-2.19.0.jar [8] - lib/commons-logging-commons-logging-1.3.5.jar [10] -- lib/io.github.merlimat.slog-slog-0.9.7.jar [64] +- lib/io.github.merlimat.slog-slog-0.9.8.jar [64] - lib/io.netty-netty-buffer-4.2.12.Final.jar [11] - lib/io.netty-netty-codec-base-4.2.12.Final.jar [11] - lib/io.netty-netty-codec-compression-4.2.12.Final.jar [11] @@ -423,7 +423,7 @@ Apache Software License, Version 2. [61] Source available at https://github.com/apache/commons-text/tree/rel/commons-text-1.13.1 [62] Source available at https://github.com/apache/commons-beanutils/tree/rel/commons-beanutils-1.11.0 [63] Source available at https://github.com/googleapis/sdk-platform-java/tree/v2.53.0/api-common-java -[64] Source available at https://github.com/merlimat/slog/tree/v0.9.7 +[64] Source available at https://github.com/merlimat/slog/tree/v0.9.8 ------------------------------------------------------------------------------------ lib/io.netty-netty-codec-base-4.2.12.Final.jar bundles some 3rd party dependencies diff --git a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt index dd0d123b0ae..8b39fb9a5dd 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt @@ -216,7 +216,7 @@ Apache Software License, Version 2. - lib/commons-codec-commons-codec-1.18.0.jar [6] - lib/commons-io-commons-io-2.19.0.jar [8] - lib/commons-logging-commons-logging-1.3.5.jar [10] -- lib/io.github.merlimat.slog-slog-0.9.7.jar [59] +- lib/io.github.merlimat.slog-slog-0.9.8.jar [59] - lib/io.netty-netty-buffer-4.2.12.Final.jar [11] - lib/io.netty-netty-codec-base-4.2.12.Final.jar [11] - lib/io.netty-netty-common-4.2.12.Final.jar [11] @@ -356,7 +356,7 @@ Apache Software License, Version 2. [56] Source available at https://github.com/apache/commons-text/tree/rel/commons-text-1.13.1 [57] Source available at https://github.com/apache/commons-beanutils/tree/rel/commons-beanutils-1.11.0 [58] Source available at https://github.com/googleapis/sdk-platform-java/tree/v2.53.0/api-common-java -[59] Source available at https://github.com/merlimat/slog/tree/v0.9.7 +[59] Source available at https://github.com/merlimat/slog/tree/v0.9.8 ------------------------------------------------------------------------------------ lib/io.netty-netty-codec-base-4.2.12.Final.jar bundles some 3rd party dependencies diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt index b2e9d676dc0..362493a0515 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt @@ -216,7 +216,7 @@ Apache Software License, Version 2. - lib/commons-codec-commons-codec-1.18.0.jar [6] - lib/commons-io-commons-io-2.19.0.jar [8] - lib/commons-logging-commons-logging-1.3.5.jar [10] -- lib/io.github.merlimat.slog-slog-0.9.7.jar [63] +- lib/io.github.merlimat.slog-slog-0.9.8.jar [63] - lib/io.netty-netty-buffer-4.2.12.Final.jar [11] - lib/io.netty-netty-codec-base-4.2.12.Final.jar [11] - lib/io.netty-netty-codec-compression-4.2.12.Final.jar [11] @@ -418,7 +418,7 @@ Apache Software License, Version 2. [60] Source available at https://github.com/apache/commons-text/tree/rel/commons-text-1.13.1 [61] Source available at https://github.com/apache/commons-beanutils/tree/rel/commons-beanutils-1.11.0 [62] Source available at https://github.com/googleapis/sdk-platform-java/tree/v2.53.0/api-common-java -[63] Source available at https://github.com/merlimat/slog/tree/v0.9.7 +[63] Source available at https://github.com/merlimat/slog/tree/v0.9.8 ------------------------------------------------------------------------------------ lib/io.netty-netty-codec-base-4.2.12.Final.jar bundles some 3rd party dependencies diff --git a/pom.xml b/pom.xml index 3de84676b52..d18f145ed76 100644 --- a/pom.xml +++ b/pom.xml @@ -192,7 +192,7 @@ 9.9.3 3.3.0 2.0.12 - 0.9.7 + 0.9.8 2.0 4.6.0 2.43.0