You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2018/01/31 21:13:40 UTC

[bookkeeper] branch master updated: BP-26 (task-2): make distributedlog modules be able to be built in bookkeeper repo

This is an automated email from the ASF dual-hosted git repository.

sijie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 5cd311c  BP-26 (task-2): make distributedlog modules be able to be built in bookkeeper repo
5cd311c is described below

commit 5cd311c7bd72273e116533d0c438254b881dd653
Author: Sijie Guo <si...@apache.org>
AuthorDate: Wed Jan 31 13:13:33 2018 -0800

    BP-26 (task-2): make distributedlog modules be able to be built in bookkeeper repo
    
    Descriptions of the changes in this PR:
    
    This is the second sub-task of #1024. This change is based on [BP-26/distributedlog_merge](https://github.com/apache/bookkeeper/tree/BP-26/distributedlog_merge) branch (created at #1068).
    
    This change includes:
    
    - move the findbugs.xml and checkstyle/suppression.xml from distributedlog-build-tools to bookkeeper buildtools module.
    - mockito: dlog is using mockito 1 while bk is using mockito 2. this change make dlog modules use the mockito dependencies defined at root pom.xml.
    - change the description from "Apache DistributedLog" to "Apache BookKeeper :: DistributedLog".
    - change the versions from `0.6.0-SNAPSHOT` to `4.7.0-SNAPSHOT`.
    - fix the compilation issues with current latest bk
    
    This PR *DOESNT* change the group id of distributedlog modules. They will remain using "org.apache.distributedlog" as the group id.
    
    Author: Sijie Guo <si...@apache.org>
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>, Jia Zhai <None>, Yiming Zang <yz...@gmail.com>
    
    This closes #1069 from sijie/compile_distributedlog
---
 .../resources/distributedlog/findbugsExclude.xml   | 181 +++++++++++++++
 .../main/resources/distributedlog/suppressions.xml |  35 +++
 dev/docker/Dockerfile                              |   5 +-
 pom.xml                                            |   5 +-
 stream/distributedlog/common/pom.xml               |  47 +---
 .../common/config/ConcurrentBaseConfiguration.java |   1 +
 .../config/ConcurrentConstConfiguration.java       |   1 +
 .../common/concurrent/TestFutureUtils.java         |  15 +-
 stream/distributedlog/core/pom.xml                 |  44 +---
 .../org/apache/bookkeeper/client/LedgerReader.java |   7 +-
 .../distributedlog/bk/SimpleLedgerAllocator.java   |   2 +-
 .../config/DynamicDistributedLogConfiguration.java |   6 +-
 .../distributedlog/impl/BKNamespaceDriver.java     |   7 +-
 .../distributedlog/impl/ZKMetadataAccessor.java    |   4 +-
 .../BKLogSegmentRandomAccessEntryReader.java       |   4 +-
 .../distributedlog/namespace/NamespaceDriver.java  |   7 +-
 .../distributedlog/tools/DistributedLogTool.java   |   7 +-
 .../org/apache/distributedlog/DLMTestUtil.java     |  12 +-
 .../TestBKDistributedLogManager.java               |   5 +-
 .../TestDistributedLogConfiguration.java           |   1 +
 .../metadata/TestZKLogStreamMetadataStore.java     |  16 +-
 stream/distributedlog/io/dlfs/pom.xml              |  30 +--
 .../org/apache/distributedlog/fs/DLFileSystem.java |   2 +-
 stream/distributedlog/io/pom.xml                   |   4 +-
 stream/distributedlog/pom.xml                      | 242 +++++++++++++++++++++
 stream/distributedlog/protocol/pom.xml             |  40 +---
 26 files changed, 545 insertions(+), 185 deletions(-)

diff --git a/buildtools/src/main/resources/distributedlog/findbugsExclude.xml b/buildtools/src/main/resources/distributedlog/findbugsExclude.xml
new file mode 100644
index 0000000..7b2b80b
--- /dev/null
+++ b/buildtools/src/main/resources/distributedlog/findbugsExclude.xml
@@ -0,0 +1,181 @@
+<!--
+    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.
+//-->
+<FindBugsFilter>
+  <Match>
+    <!-- generated code, we can't be held responsible for findbugs in it //-->
+    <Class name="~org\.apache\.distributedlog\.tests\.generated.*" />
+  </Match>
+  <Match>
+    <!-- generated code, we can't be held responsible for findbugs in it //-->
+    <Class name="~org\.apache\.distributedlog\.thrift.*" />
+  </Match>
+  <!-- distributedlog-benchmark -->
+  <Match>
+    <!-- generated code, we can't be held responsible for findbugs in it //-->
+    <Class name="~org\.apache\.distributedlog\.benchmark\.thrift.*" />
+  </Match>
+  <!-- distributedlog-common -->
+  <Match>
+    <Class name="org.apache.distributedlog.common.concurrent.FutureUtils"/>
+    <Bug pattern="NP_NULL_PARAM_DEREF_NONVIRTUAL" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.common.concurrent.FutureUtils$2"/>
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.common.concurrent.FutureUtils"/>
+    <Method name="Void" />
+    <Bug pattern="NM_METHOD_NAMING_CONVENTION" />
+  </Match>
+  <!-- distributedlog-core -->
+  <Match>
+    <!-- it is safe to store external bytes reference here. //-->
+    <Class name="org.apache.distributedlog.Entry$Builder" />
+    <Method name="setData" />
+    <Bug pattern="EI_EXPOSE_REP2" />
+  </Match>
+  <Match>
+    <!-- it is safe to store external bytes reference here. //-->
+    <Class name="org.apache.distributedlog.Entry" />
+    <Method name="getRawData" />
+    <Bug pattern="EI_EXPOSE_REP" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.BKAsyncLogReader" />
+    <Method name="run" />
+    <Bug pattern="JLM_JSR166_UTILCONCURRENT_MONITORENTER" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.BKLogReadHandler$1" />
+    <Method name="onSuccess" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.BookKeeperClient$2" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.ReadUtils" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.ReadUtils$2" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.auditor.DLAuditor$2" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.auditor.DLAuditor$8" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.bk.SimpleLedgerAllocator$4" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.bk.SimpleLedgerAllocator$4$1" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.bk.SimpleLedgerAllocator$5" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.impl.acl.ZKAccessControl$4" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.impl.acl.ZKAccessControlManager$1" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.impl.acl.ZKAccessControlManager$1$1" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore$1$1" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.lock.ZKSessionLock" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.lock.ZKSessionLock$12" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.lock.ZKSessionLock$13$1" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.util.Utils" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.util.Utils$6" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <!-- distributedlog-protocol -->
+  <Match>
+    <!-- it is safe to store external bytes reference here. //-->
+    <Class name="org.apache.distributedlog.LogRecord" />
+    <Bug pattern="EI_EXPOSE_REP2" />
+  </Match>
+  <Match>
+    <!-- it is safe to store external bytes reference here. //-->
+    <Class name="org.apache.distributedlog.LogRecord" />
+    <Method name="getPayload" />
+    <Bug pattern="EI_EXPOSE_REP" />
+  </Match>
+  <!-- distributedlog-proxy-server -->
+  <Match>
+    <!-- generated code, we can't be held responsible for findbugs in it //-->
+    <Class name="~org\.apache\.distributedlog\.service\.placement\.thrift.*" />
+  </Match>
+  <Match>
+    <!-- it is safe to cast exception here. //-->
+    <Class name="org.apache.distributedlog.service.DistributedLogServiceImpl$Stream$2" />
+    <Method name="onFailure" />
+    <Bug pattern="BC_UNCONFIRMED_CAST" />
+  </Match>
+  <Match>
+    <!-- it is safe to cast exception here. //-->
+    <Class name="org.apache.distributedlog.service.stream.BulkWriteOp" />
+    <Method name="isDefiniteFailure" />
+    <Bug pattern="BC_IMPOSSIBLE_INSTANCEOF" />
+  </Match>
+  <!-- distributedlog-messaging -->
+  <Match>
+    <!-- generated code, we can't be held responsible for findbugs in it //-->
+    <Class name="~.*\.TransformedRecord" />
+  </Match>
+  <Match>
+    <!-- it is safe to store external bytes reference here. //-->
+    <Class name="org.apache.distributedlog.messaging.PartitionedMultiWriter" />
+    <Bug pattern="EI_EXPOSE_REP2" />
+  </Match>
+  <Match>
+    <!-- it is safe to store external bytes reference here. //-->
+    <Class name="org.apache.distributedlog.messaging.RRMultiWriter" />
+    <Bug pattern="EI_EXPOSE_REP2" />
+  </Match>
+</FindBugsFilter>
diff --git a/buildtools/src/main/resources/distributedlog/suppressions.xml b/buildtools/src/main/resources/distributedlog/suppressions.xml
new file mode 100644
index 0000000..6ee57b1
--- /dev/null
+++ b/buildtools/src/main/resources/distributedlog/suppressions.xml
@@ -0,0 +1,35 @@
+<?xml version="1.0"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<!DOCTYPE suppressions PUBLIC
+"-//Puppy Crawl//DTD Suppressions 1.1//EN"
+"http://www.puppycrawl.com/dtds/suppressions_1_1.dtd">
+
+<suppressions>
+  <suppress checks="JavadocPackage" files=".*[\\/]src[\\/]test[\\/].*"/>
+  <suppress checks="JavadocPackage" files=".*[\\/]maven-archetypes[\\/].*"/>
+  <suppress checks="JavadocPackage" files=".*[\\/]examples[\\/].*"/>
+  <suppress checks="AvoidStarImport" files=".*[\\/]distributedlog[\\/].*"/>
+  <suppress checks="StaticVariableName" files=".*[\\/]distributedlog[\\/].*"/>
+  <suppress checks="LocalVariableName" files=".*[\\/]distributedlog[\\/].*"/>
+  <suppress checks="LocalFinalVariableName" files=".*[\\/]distributedlog[\\/].*"/>
+  <suppress checks="MemberName" files=".*[\\/]distributedlog[\\/].*"/>
+
+  <!-- suppress all checks in the generated directories -->
+  <suppress checks=".*" files=".*[\\/]distributedlog\.thrift[\\/].*"/>
+  <suppress checks=".*" files=".+[\\/]generated[\\/].+\.java" />
+  <suppress checks=".*" files=".+[\\/]generated-sources[\\/].+\.java" />
+  <suppress checks=".*" files=".+[\\/]generated-test-sources[\\/].+\.java" />
+</suppressions>
+
diff --git a/dev/docker/Dockerfile b/dev/docker/Dockerfile
index f803b96..228c234 100644
--- a/dev/docker/Dockerfile
+++ b/dev/docker/Dockerfile
@@ -17,4 +17,7 @@
 # under the License.
 #
 
-FROM maven:3.5.0-jdk-8
+FROM maven:3.5.0-jdk-9
+
+RUN apt-get update
+RUN apt-get install -y g++ cmake
diff --git a/pom.xml b/pom.xml
index 8565fef..0742417 100644
--- a/pom.xml
+++ b/pom.xml
@@ -64,6 +64,7 @@
     <module>tests</module>
     <module>bookkeeper-dist</module>
     <module>microbenchmarks</module>
+    <module>stream/distributedlog</module>
   </modules>
   <mailingLists>
     <mailingList>
@@ -110,7 +111,7 @@
     <jmh.version>1.19</jmh.version>
     <junit.version>4.12</junit.version>
     <lombok.version>1.16.18</lombok.version>
-    <mockito-core.version>2.13.0</mockito-core.version>
+    <mockito.version>2.13.0</mockito.version>
     <netty.version>4.1.12.Final</netty.version>
     <netty-boringssl.version>2.0.3.Final</netty-boringssl.version>
     <powermock.version>2.0.0-beta.5</powermock.version>
@@ -181,7 +182,7 @@
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-core</artifactId>
-      <version>${mockito-core.version}</version>
+      <version>${mockito.version}</version>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/stream/distributedlog/common/pom.xml b/stream/distributedlog/common/pom.xml
index 19d3109..d550150 100644
--- a/stream/distributedlog/common/pom.xml
+++ b/stream/distributedlog/common/pom.xml
@@ -20,15 +20,15 @@
   <parent>
     <groupId>org.apache.distributedlog</groupId>
     <artifactId>distributedlog</artifactId>
-    <version>0.6.0-SNAPSHOT</version>
+    <version>4.7.0-SNAPSHOT</version>
   </parent>
   <artifactId>distributedlog-common</artifactId>
-  <name>Apache DistributedLog :: Common</name>
+  <name>Apache BookKeeper :: DistributedLog :: Common</name>
   <dependencies>
     <dependency>
       <groupId>org.apache.bookkeeper.stats</groupId>
       <artifactId>bookkeeper-stats-api</artifactId>
-      <version>${bookkeeper.version}</version>
+      <version>${project.parent.version}</version>
       <exclusions>
         <exclusion>
           <groupId>org.slf4j</groupId>
@@ -37,17 +37,6 @@
       </exclusions>
     </dependency>
     <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-      <version>${slf4j.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.projectlombok</groupId>
-      <artifactId>lombok</artifactId>
-      <version>${lombok.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
       <groupId>org.inferred</groupId>
       <artifactId>freebuilder</artifactId>
       <version>${freebuilder.version}</version>
@@ -78,24 +67,6 @@
       <version>${lz4.version}</version>
     </dependency>
     <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <version>${junit.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-log4j12</artifactId>
-      <version>${slf4j.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-core</artifactId>
-      <version>${mockito.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.jmock</groupId>
       <artifactId>jmock</artifactId>
       <version>${jmock.version}</version>
@@ -105,8 +76,8 @@
   <build>
     <plugins>
       <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>findbugs-maven-plugin</artifactId>
+        <groupId>com.github.spotbugs</groupId>
+        <artifactId>spotbugs-maven-plugin</artifactId>
       </plugin>
       <plugin>
         <artifactId>maven-compiler-plugin</artifactId>
@@ -146,13 +117,13 @@
             <version>${puppycrawl.checkstyle.version}</version>
           </dependency>
           <dependency>
-            <groupId>org.apache.distributedlog</groupId>
-            <artifactId>distributedlog-build-tools</artifactId>
-            <version>${project.version}</version>
+            <groupId>org.apache.bookkeeper</groupId>
+            <artifactId>buildtools</artifactId>
+            <version>${project.parent.version}</version>
           </dependency>
         </dependencies>
         <configuration>
-          <configLocation>distributedlog/checkstyle.xml</configLocation>
+          <configLocation>bookkeeper/checkstyle.xml</configLocation>
           <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
           <consoleOutput>true</consoleOutput>
           <failOnViolation>true</failOnViolation>
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConcurrentBaseConfiguration.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConcurrentBaseConfiguration.java
index 83e8e0e..b390431 100644
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConcurrentBaseConfiguration.java
+++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConcurrentBaseConfiguration.java
@@ -30,6 +30,7 @@ import org.apache.commons.configuration.AbstractConfiguration;
  * normally combines all properties with the same key into one list property automatically.
  * This class simply overwrites any existing mapping.
  */
+@SuppressWarnings("unchecked")
 public class ConcurrentBaseConfiguration extends AbstractConfiguration {
 
     private final ConcurrentHashMap<String, Object> map;
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConcurrentConstConfiguration.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConcurrentConstConfiguration.java
index 1131409..2b99942 100644
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConcurrentConstConfiguration.java
+++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/config/ConcurrentConstConfiguration.java
@@ -24,6 +24,7 @@ import org.apache.commons.configuration.Configuration;
 /**
  * Invariant thread-safe view of some configuration.
  */
+@SuppressWarnings("unchecked")
 public class ConcurrentConstConfiguration extends ConcurrentBaseConfiguration {
     public ConcurrentConstConfiguration(Configuration conf) {
         checkNotNull(conf);
diff --git a/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/concurrent/TestFutureUtils.java b/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/concurrent/TestFutureUtils.java
index a887c59..fecefbc 100644
--- a/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/concurrent/TestFutureUtils.java
+++ b/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/concurrent/TestFutureUtils.java
@@ -22,11 +22,9 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.anyObject;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.isA;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
@@ -175,7 +173,7 @@ public class TestFutureUtils {
         assertFalse(withinFuture.isCancelled());
         assertFalse(withinFuture.isCompletedExceptionally());
         verify(scheduler, times(0))
-            .schedule(eq(1234L), isA(Runnable.class), eq(10), eq(TimeUnit.MILLISECONDS));
+            .schedule(eq(1234L), any(Runnable.class), eq(10), eq(TimeUnit.MILLISECONDS));
     }
 
     @Test
@@ -194,14 +192,14 @@ public class TestFutureUtils {
         assertFalse(withinFuture.isCancelled());
         assertFalse(withinFuture.isCompletedExceptionally());
         verify(scheduler, times(0))
-            .schedule(eq(1234L), isA(Runnable.class), eq(10), eq(TimeUnit.MILLISECONDS));
+            .schedule(eq(1234L), any(Runnable.class), eq(10), eq(TimeUnit.MILLISECONDS));
     }
 
     @Test
     public void testWithinCompleteBeforeTimeout() throws Exception {
         OrderedScheduler scheduler = mock(OrderedScheduler.class);
         ScheduledFuture<?> scheduledFuture = mock(ScheduledFuture.class);
-        when(scheduler.schedule(anyObject(), any(Runnable.class), anyLong(), any(TimeUnit.class)))
+        when(scheduler.schedule(any(), any(Runnable.class), anyLong(), any(TimeUnit.class)))
             .thenAnswer(invocationOnMock -> scheduledFuture);
         CompletableFuture<Long> newFuture = FutureUtils.createFuture();
         CompletableFuture<Long> withinFuture = FutureUtils.within(
@@ -276,6 +274,7 @@ public class TestFutureUtils {
         ensureLatch.await();
     }
 
+    @SuppressWarnings("unchecked")
     @Test
     public void testRescueSuccess() throws Exception {
         CompletableFuture<Long> underlyFuture = FutureUtils.createFuture();
diff --git a/stream/distributedlog/core/pom.xml b/stream/distributedlog/core/pom.xml
index aa55c2e..9f16736 100644
--- a/stream/distributedlog/core/pom.xml
+++ b/stream/distributedlog/core/pom.xml
@@ -20,10 +20,10 @@
   <parent>
     <groupId>org.apache.distributedlog</groupId>
     <artifactId>distributedlog</artifactId>
-    <version>0.6.0-SNAPSHOT</version>
+    <version>4.7.0-SNAPSHOT</version>
   </parent>
   <artifactId>distributedlog-core</artifactId>
-  <name>Apache DistributedLog :: Core Library</name>
+  <name>Apache BookKeeper :: DistributedLog :: Core Library</name>
   <dependencies>
     <dependency>
       <groupId>org.apache.distributedlog</groupId>
@@ -62,7 +62,7 @@
     <dependency>
       <groupId>org.apache.bookkeeper</groupId>
       <artifactId>bookkeeper-server</artifactId>
-      <version>${bookkeeper.version}</version>
+      <version>${project.parent.version}</version>
       <type>jar</type>
       <exclusions>
         <exclusion>
@@ -88,36 +88,12 @@
       </exclusions>
     </dependency>
     <dependency>
-      <groupId>org.projectlombok</groupId>
-      <artifactId>lombok</artifactId>
-      <version>${lombok.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <version>${junit.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.jmock</groupId>
       <artifactId>jmock</artifactId>
       <version>${jmock.version}</version>
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-log4j12</artifactId>
-      <version>${slf4j.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-core</artifactId>
-      <version>${mockito.version}</version>
-      <scope>test</scope>
-    </dependency> 
-    <dependency>
       <groupId>org.apache.distributedlog</groupId>
       <artifactId>distributedlog-common</artifactId>
       <version>${project.parent.version}</version>
@@ -161,8 +137,8 @@
         </configuration>
       </plugin>
       <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>findbugs-maven-plugin</artifactId>
+        <groupId>com.github.spotbugs</groupId>
+        <artifactId>spotbugs-maven-plugin</artifactId>
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
@@ -474,13 +450,13 @@
             <version>${puppycrawl.checkstyle.version}</version>
           </dependency>
           <dependency>
-            <groupId>org.apache.distributedlog</groupId>
-            <artifactId>distributedlog-build-tools</artifactId>
-            <version>${project.version}</version>
+            <groupId>org.apache.bookkeeper</groupId>
+            <artifactId>buildtools</artifactId>
+            <version>${project.parent.version}</version>
           </dependency>
         </dependencies>
         <configuration>
-          <configLocation>distributedlog/checkstyle.xml</configLocation>
+          <configLocation>bookkeeper/checkstyle.xml</configLocation>
           <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
           <consoleOutput>true</consoleOutput>
           <failOnViolation>true</failOnViolation>
@@ -490,7 +466,7 @@
         </configuration>
         <executions>
           <execution>
-            <phase>test-compile</phase>
+            <phase>validate</phase>
             <goals>
               <goal>check</goal>
             </goals>
diff --git a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java
index 6c6bd4a..60d9096 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java
@@ -36,6 +36,7 @@ import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookieClient;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
+import org.apache.bookkeeper.proto.checksum.DigestManager;
 
 /**
  * Reader used for DL tools to read entries.
@@ -184,12 +185,12 @@ public class LedgerReader {
             }
 
 
-            if (LedgerHandle.INVALID_ENTRY_ID >= recoveryData.lastAddConfirmed) {
+            if (LedgerHandle.INVALID_ENTRY_ID >= recoveryData.getLastAddConfirmed()) {
                 callback.operationComplete(BKException.Code.OK, resultList);
                 return;
             }
 
-            long entryId = recoveryData.lastAddConfirmed;
+            long entryId = recoveryData.getLastAddConfirmed();
             PendingReadOp op = new PendingReadOp(lh, lh.bk.scheduler, entryId, entryId, false);
             op.future().whenComplete(readListener);
             op.submit();
@@ -211,7 +212,7 @@ public class LedgerReader {
             } else {
                 try {
                     DigestManager.RecoveryData data = lh.macManager.verifyDigestAndReturnLastConfirmed(buffer);
-                    rr = new ReadResult<Long>(eid1, BKException.Code.OK, data.lastAddConfirmed, bookieAddress);
+                    rr = new ReadResult<Long>(eid1, BKException.Code.OK, data.getLastAddConfirmed(), bookieAddress);
                 } catch (BKException.BKDigestMatchException e) {
                     rr = new ReadResult<Long>(eid1, BKException.Code.DigestMatchException, null, bookieAddress);
                 }
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 d87f557..7a9fa58 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
@@ -440,7 +440,7 @@ public class SimpleLedgerAllocator implements LedgerAllocator, FutureEventListen
         }
         if (!cleanup) {
             LOG.info("Abort ledger allocator without cleaning up on {}.", allocatePath);
-            closePromise.complete(null);
+            FutureUtils.complete(closePromise, null);
             return closePromise;
         }
         cleanupAndClose(closePromise);
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java
index 88c114c..4e2755f 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java
@@ -19,18 +19,14 @@ package org.apache.distributedlog.config;
 
 import static org.apache.distributedlog.DistributedLogConfiguration.*;
 
-
-
 import org.apache.distributedlog.DistributedLogConfiguration;
-
 import org.apache.distributedlog.bk.QuorumConfig;
 import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
 
-
-
 /**
  * Whitelist dynamic configuration by adding an accessor to this class.
  */
+@SuppressWarnings("unchecked")
 public class DynamicDistributedLogConfiguration extends ConcurrentBaseConfiguration {
 
     private final ConcurrentBaseConfiguration defaultConfig;
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 30c42cb..293ac2a 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
@@ -51,7 +51,6 @@ import org.apache.distributedlog.ZooKeeperClientBuilder;
 
 import org.apache.distributedlog.acl.AccessControlManager;
 import org.apache.distributedlog.acl.DefaultAccessControlManager;
-import org.apache.distributedlog.api.MetadataAccessor;
 import org.apache.distributedlog.api.subscription.SubscriptionsStore;
 import org.apache.distributedlog.bk.LedgerAllocator;
 import org.apache.distributedlog.bk.LedgerAllocatorUtils;
@@ -83,9 +82,6 @@ import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
-
-
 /**
  * Manager for ZooKeeper/BookKeeper based namespace.
  */
@@ -520,8 +516,9 @@ public class BKNamespaceDriver implements NamespaceDriver {
     // Legacy Intefaces
     //
 
+    @SuppressWarnings("deprecation")
     @Override
-    public MetadataAccessor getMetadataAccessor(String streamName)
+    public org.apache.distributedlog.api.MetadataAccessor getMetadataAccessor(String streamName)
             throws InvalidStreamNameException, IOException {
         if (getBkdlConfig().isFederatedNamespace()) {
             throw new UnsupportedOperationException();
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 ad86b57..481d398 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
@@ -30,7 +30,6 @@ import org.apache.bookkeeper.zookeeper.RetryPolicy;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClientBuilder;
-import org.apache.distributedlog.api.MetadataAccessor;
 import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.AlreadyClosedException;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
@@ -46,7 +45,8 @@ import org.slf4j.LoggerFactory;
 /**
  * access to ZKMetadata.
  */
-public class ZKMetadataAccessor implements MetadataAccessor {
+@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<Void> closePromise;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
index 3d50231..62cfb98 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
@@ -33,9 +33,6 @@ import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.BKTransmitException;
 import org.apache.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
 
-
-
-
 /**
  * BookKeeper ledger based random access entry reader.
  */
@@ -85,6 +82,7 @@ class BKLogSegmentRandomAccessEntryReader implements
                 .buildReader();
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> entries, Object ctx) {
         CompletableFuture<List<Entry.Reader>> promise = (CompletableFuture<List<Entry.Reader>>) ctx;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriver.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriver.java
index 432b22a..84f8ff3 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriver.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriver.java
@@ -24,7 +24,6 @@ import org.apache.bookkeeper.feature.FeatureProvider;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.acl.AccessControlManager;
-import org.apache.distributedlog.api.MetadataAccessor;
 import org.apache.distributedlog.api.subscription.SubscriptionsStore;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.InvalidStreamNameException;
@@ -34,9 +33,6 @@ import org.apache.distributedlog.metadata.LogMetadataStore;
 import org.apache.distributedlog.metadata.LogStreamMetadataStore;
 import org.apache.distributedlog.util.OrderedScheduler;
 
-
-
-
 /**
  * Manager to manage all the stores required by a namespace.
  */
@@ -128,7 +124,8 @@ public interface NamespaceDriver extends Closeable {
      * @param streamName name of log stream.
      * @return metadata accessor for log stream {@code streamName}.
      */
-    MetadataAccessor getMetadataAccessor(String streamName)
+    @SuppressWarnings("deprecation")
+    org.apache.distributedlog.api.MetadataAccessor getMetadataAccessor(String streamName)
             throws InvalidStreamNameException, IOException;
 
     /**
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 04c0810..8ae05fe 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
@@ -86,7 +86,6 @@ import org.apache.distributedlog.api.AsyncLogReader;
 import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.api.LogReader;
-import org.apache.distributedlog.api.MetadataAccessor;
 import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.auditor.DLAuditor;
@@ -106,9 +105,6 @@ import org.apache.distributedlog.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
-
-
 /**
  *DistributedLogTool.
 */
@@ -470,6 +466,7 @@ import org.slf4j.LoggerFactory;
             return 0;
         }
 
+        @SuppressWarnings("deprecation")
         protected void printStreams(Namespace namespace) throws Exception {
             Iterator<String> streams = namespace.getLogs();
             System.out.println("Streams under " + getUri() + " : ");
@@ -480,7 +477,7 @@ import org.slf4j.LoggerFactory;
                 if (!printMetadata) {
                     continue;
                 }
-                MetadataAccessor accessor =
+                org.apache.distributedlog.api.MetadataAccessor accessor =
                         namespace.getNamespaceDriver().getMetadataAccessor(streamName);
                 byte[] metadata = accessor.getMetadata();
                 if (null == metadata || metadata.length == 0) {
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 7e1464c..986ae9e 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
@@ -41,7 +41,6 @@ import org.apache.bookkeeper.versioning.Version;
 import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.api.LogReader;
-import org.apache.distributedlog.api.MetadataAccessor;
 import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.common.util.PermitLimiter;
@@ -51,12 +50,9 @@ import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
 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.
@@ -100,9 +96,11 @@ public class DLMTestUtil {
         return namespace.openLog(name);
     }
 
-    static MetadataAccessor createNewMetadataAccessor(DistributedLogConfiguration conf,
-                                                      String name,
-                                                      URI uri) throws Exception {
+    @SuppressWarnings("deprecation")
+    static org.apache.distributedlog.api.MetadataAccessor createNewMetadataAccessor(
+            DistributedLogConfiguration conf,
+            String name,
+            URI uri) throws Exception {
         // TODO: Metadata Accessor seems to be a legacy object which only used by kestrel
         //       (we might consider deprecating this)
         Namespace namespace = NamespaceBuilder.newBuilder()
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 c44df2f..6a70313 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
@@ -43,7 +43,6 @@ import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.api.LogWriter;
-import org.apache.distributedlog.api.MetadataAccessor;
 import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.api.subscription.SubscriptionsStore;
@@ -492,10 +491,12 @@ public class TestBKDistributedLogManager extends TestDistributedLogBase {
         namespace.close();
     }
 
+    @SuppressWarnings("deprecation")
     @Test(timeout = 60000)
     public void testMetadataAccessor() throws Exception {
         String name = "distrlog-metadata-accessor";
-        MetadataAccessor metadata = DLMTestUtil.createNewMetadataAccessor(conf, name, createDLMURI("/" + name));
+        org.apache.distributedlog.api.MetadataAccessor metadata =
+            DLMTestUtil.createNewMetadataAccessor(conf, name, createDLMURI("/" + name));
         assertEquals(name, metadata.getStreamName());
         metadata.createOrUpdateMetadata(name.getBytes());
         assertEquals(name, new String(metadata.getMetadata()));
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java
index c0eecdf..e2a9705 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java
@@ -105,6 +105,7 @@ public class TestDistributedLogConfiguration {
         assertEquals(TestDNSResolver.class, conf3.getEnsemblePlacementDnsResolverClass());
     }
 
+    @SuppressWarnings("deprecation")
     @Test(timeout = 200000)
     public void validateConfiguration(){
         boolean exceptionThrown = false;
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 4aa832a..baeaab9 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
@@ -25,10 +25,9 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyObject;
-import static org.mockito.Matchers.anyString;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -48,7 +47,6 @@ import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.TestZooKeeperClientBuilder;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClusterTestCase;
-import org.apache.distributedlog.api.MetadataAccessor;
 import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.common.concurrent.FutureUtils;
@@ -396,6 +394,7 @@ public class TestZKLogStreamMetadataStore extends ZooKeeperClusterTestCase {
         Utils.ioResult(getLog(uri, logName, logIdentifier, zkc, true, false));
     }
 
+    @SuppressWarnings("deprecation")
     @Test(timeout = 60000)
     public void testCreateLogMetadataWithCustomMetadata() throws Exception {
         String logName = testName.getMethodName();
@@ -409,7 +408,8 @@ public class TestZKLogStreamMetadataStore extends ZooKeeperClusterTestCase {
             .uri(uri)
             .build();
 
-        MetadataAccessor accessor = namespace.getNamespaceDriver().getMetadataAccessor(logName);
+        org.apache.distributedlog.api.MetadataAccessor accessor =
+            namespace.getNamespaceDriver().getMetadataAccessor(logName);
         accessor.createOrUpdateMetadata(logName.getBytes("UTF-8"));
         accessor.close();
 
@@ -438,7 +438,7 @@ public class TestZKLogStreamMetadataStore extends ZooKeeperClusterTestCase {
             Children2Callback callback = (Children2Callback) invocationOnMock.getArguments()[2];
             callback.processResult(Code.BADVERSION.intValue(), path, null, null, null);
             return null;
-        }).when(mockZk).getChildren(anyString(), anyBoolean(), any(Children2Callback.class), anyObject());
+        }).when(mockZk).getChildren(anyString(), anyBoolean(), any(Children2Callback.class), any());
 
         String logSegmentsPath = LogMetadata.getLogSegmentsPath(uri, logName, logIdentifier);
         try {
@@ -508,7 +508,7 @@ public class TestZKLogStreamMetadataStore extends ZooKeeperClusterTestCase {
             StatCallback callback = (StatCallback) invocationOnMock.getArguments()[2];
             callback.processResult(Code.BADVERSION.intValue(), path, null, null);
             return null;
-        }).when(mockZk).exists(anyString(), anyBoolean(), any(StatCallback.class), anyObject());
+        }).when(mockZk).exists(anyString(), anyBoolean(), any(StatCallback.class), any());
 
         try {
             FutureUtils.result(getMissingPaths(mockZkc, uri, "path/to/log"));
diff --git a/stream/distributedlog/io/dlfs/pom.xml b/stream/distributedlog/io/dlfs/pom.xml
index 775f07b..b6df19b 100644
--- a/stream/distributedlog/io/dlfs/pom.xml
+++ b/stream/distributedlog/io/dlfs/pom.xml
@@ -21,12 +21,12 @@
   <parent>
     <artifactId>distributedlog</artifactId>
     <groupId>org.apache.distributedlog</groupId>
-    <version>0.6.0-SNAPSHOT</version>
+    <version>4.7.0-SNAPSHOT</version>
     <relativePath>../..</relativePath>
   </parent>
   <groupId>org.apache.distributedlog</groupId>
   <artifactId>dlfs</artifactId>
-  <name>Apache DistributedLog :: IO :: FileSystem</name>
+  <name>Apache BookKeeper :: DistributedLog :: IO :: FileSystem</name>
   <url>http://maven.apache.org</url>
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
@@ -34,12 +34,6 @@
   </properties>
   <dependencies>
     <dependency>
-      <groupId>org.projectlombok</groupId>
-      <artifactId>lombok</artifactId>
-      <version>${lombok.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
       <groupId>org.apache.distributedlog</groupId>
       <artifactId>distributedlog-core</artifactId>
       <version>${project.parent.version}</version>
@@ -60,12 +54,6 @@
       </exclusions>
     </dependency>
     <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <version>${junit.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.apache.distributedlog</groupId>
       <artifactId>distributedlog-core</artifactId>
       <version>${project.parent.version}</version>
@@ -76,8 +64,8 @@
   <build>
     <plugins>
       <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>findbugs-maven-plugin</artifactId>
+        <groupId>com.github.spotbugs</groupId>
+        <artifactId>spotbugs-maven-plugin</artifactId>
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
@@ -90,13 +78,13 @@
             <version>${puppycrawl.checkstyle.version}</version>
           </dependency>
           <dependency>
-            <groupId>org.apache.distributedlog</groupId>
-            <artifactId>distributedlog-build-tools</artifactId>
-            <version>${project.version}</version>
+            <groupId>org.apache.bookkeeper</groupId>
+            <artifactId>buildtools</artifactId>
+            <version>${project.parent.version}</version>
           </dependency>
         </dependencies>
         <configuration>
-          <configLocation>distributedlog/checkstyle.xml</configLocation>
+          <configLocation>bookkeeper/checkstyle.xml</configLocation>
           <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
           <consoleOutput>true</consoleOutput>
           <failOnViolation>true</failOnViolation>
@@ -105,7 +93,7 @@
         </configuration>
         <executions>
           <execution>
-            <phase>test-compile</phase>
+            <phase>validate</phase>
             <goals>
               <goal>check</goal>
             </goals>
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 0670a4a..1a056c3 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
@@ -265,7 +265,7 @@ public class DLFileSystem extends FileSystem {
                 Path childPath = new Path(path, child);
                 statusList.add(getFileStatus(childPath));
             }
-            Collections.sort(statusList, Comparator.comparing(FileStatus::getPath));
+            Collections.sort(statusList, Comparator.comparing(fileStatus -> fileStatus.getPath().getName()));
             return statusList.toArray(new FileStatus[statusList.size()]);
         } catch (LogNotFoundException e) {
             throw new FileNotFoundException(path.toString());
diff --git a/stream/distributedlog/io/pom.xml b/stream/distributedlog/io/pom.xml
index be82c40..1898b3f 100644
--- a/stream/distributedlog/io/pom.xml
+++ b/stream/distributedlog/io/pom.xml
@@ -22,12 +22,12 @@
   <parent>
     <groupId>org.apache.distributedlog</groupId>
     <artifactId>distributedlog</artifactId>
-    <version>0.6.0-SNAPSHOT</version>
+    <version>4.7.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>distributedlog-io</artifactId>
   <packaging>pom</packaging>
-  <name>Apache DistributedLog :: IO</name>
+  <name>Apache BookKeeper :: DistributedLog :: IO</name>
   <modules>
     <module>dlfs</module>
   </modules>
diff --git a/stream/distributedlog/pom.xml b/stream/distributedlog/pom.xml
new file mode 100644
index 0000000..545ebaa
--- /dev/null
+++ b/stream/distributedlog/pom.xml
@@ -0,0 +1,242 @@
+<?xml version="1.0"?>
+<!--
+   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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.bookkeeper</groupId>
+    <artifactId>bookkeeper</artifactId>
+    <version>4.7.0-SNAPSHOT</version>
+    <relativePath>../..</relativePath>
+  </parent>
+  <groupId>org.apache.distributedlog</groupId>
+  <artifactId>distributedlog</artifactId>
+  <packaging>pom</packaging>
+  <name>Apache BookKeeper :: DistributedLog :: Parent</name>
+  <description>
+    Apache DistributedLog provides a high performance replicated log service.
+  </description>
+  <inceptionYear>2016</inceptionYear>
+  <modules>
+    <module>common</module>
+    <module>protocol</module>
+    <module>core</module>
+    <module>io</module>
+  </modules>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+    <!-- dependencies -->
+    <codahale.metrics.version>3.0.1</codahale.metrics.version>
+    <commons-cli.version>1.1</commons-cli.version>
+    <commons-codec.version>1.6</commons-codec.version>
+    <commons-lang.version>2.6</commons-lang.version>
+    <commons-lang3.version>3.3.2</commons-lang3.version>
+    <curator.version>4.0.0</curator.version>
+    <finagle.version>6.34.0</finagle.version>
+    <freebuilder.version>1.12.3</freebuilder.version>
+    <guava.version>20.0</guava.version>
+    <jetty.version>9.3.11.v20160721</jetty.version>
+    <jmh.version>1.19</jmh.version>
+    <jmock.version>2.8.2</jmock.version>
+    <junit.version>4.8.1</junit.version>
+    <libthrift.version>0.5.0-1</libthrift.version>
+    <lz4.version>1.3.0</lz4.version>
+    <netty.version>4.1.12.Final</netty.version>
+    <scrooge.version>4.6.0</scrooge.version>
+    <slf4j.version>1.6.4</slf4j.version>
+    <prometheus.version>0.0.21</prometheus.version>
+    <stats-util.version>0.0.58</stats-util.version>
+    <zookeeper.version>3.5.3-beta</zookeeper.version>
+    <!-- plugin dependencies -->
+    <apache-rat-plugin.version>0.12</apache-rat-plugin.version>
+    <cobertura-maven-plugin.version>2.7</cobertura-maven-plugin.version>
+    <coveralls-maven-plugin.version>4.1.0</coveralls-maven-plugin.version>
+    <puppycrawl.checkstyle.version>6.19</puppycrawl.checkstyle.version>
+    <maven-assembly-plugin.version>2.2.1</maven-assembly-plugin.version>
+    <maven-checkstyle-plugin.version>2.17</maven-checkstyle-plugin.version>
+    <maven-compiler-plugin.version>3.1</maven-compiler-plugin.version>
+    <maven-deploy-plugin.version>2.7</maven-deploy-plugin.version>
+    <maven-jar-plugin.version>2.2</maven-jar-plugin.version>
+    <maven-javadoc-plugin.version>2.8</maven-javadoc-plugin.version>
+    <maven-shade-plugin.version>2.4.3</maven-shade-plugin.version>
+    <maven-source-plugin.version>2.2.1</maven-source-plugin.version>
+    <maven-surefire-plugin.version>2.19.1</maven-surefire-plugin.version>
+    <scrooge-maven-plugin.version>3.17.0</scrooge-maven-plugin.version>
+  </properties>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-javadoc-plugin</artifactId>
+        <version>${maven-javadoc-plugin.version}</version>
+        <configuration>
+          <!-- Avoid for missing javadoc comments to be marked as errors -->
+          <additionalparam>-Xdoclint:none -notimestamp</additionalparam>
+          <groups>
+            <group>
+              <title>Core Library</title>
+              <packages>org.apache.distributedlog:org.apache.distributedlog.annotations:org.apache.distributedlog.callback:org.apache.distributedlog.exceptions:org.apache.distributedlog.feature:org.apache.distributedlog.io:org.apache.distributedlog.lock:org.apache.distributedlog.logsegment:org.apache.distributedlog.metadata:org.apache.distributedlog.namespace:org.apache.distributedlog.net:org.apache.distributedlog.stats:org.apache.distributedlog.api.subscription</packages>
+            </group>
+          </groups>
+          <excludePackageNames>
+            org.apache.distributedlog.acl:org.apache.distributedlog.admin:org.apache.distributedlog.auditor:org.apache.distributedlog.basic:org.apache.distributedlog.benchmark*:org.apache.distributedlog.bk:org.apache.distributedlog.ownership:org.apache.distributedlog.proxy:org.apache.distributedlog.resolver:org.apache.distributedlog.service.*:org.apache.distributedlog.config:org.apache.distributedlog.function:org.apache.distributedlog.impl*:org.apache.distributedlog.injector:org.apache.d [...]
+          </excludePackageNames>
+        </configuration>
+        <executions>
+          <execution>
+            <id>aggregate</id>
+            <goals>
+              <goal>aggregate</goal>
+            </goals>
+            <phase>site</phase>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <artifactId>maven-assembly-plugin</artifactId>
+        <version>${maven-assembly-plugin.version}</version>
+        <configuration>
+          <tarLongFileMode>gnu</tarLongFileMode>
+          <descriptors>
+            <descriptor>src/assemble/src.xml</descriptor>
+          </descriptors>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>com.github.spotbugs</groupId>
+        <artifactId>spotbugs-maven-plugin</artifactId>
+        <version>${spotbugs-maven-plugin.version}</version>
+        <configuration>
+          <excludeFilterFile>${session.executionRootDirectory}/buildtools/src/main/resources/distributedlog/findbugsExclude.xml</excludeFilterFile>
+        </configuration>
+      </plugin>
+      <plugin>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>${maven-compiler-plugin.version}</version>
+        <configuration>
+          <source>1.8</source>
+          <target>1.8</target>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+        <version>${maven-source-plugin.version}</version>
+        <executions>
+          <execution>
+            <id>attach-sources</id>
+            <goals>
+              <goal>jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>${maven-surefire-plugin.version}</version>
+        <configuration>
+          <!-- only run tests when -DintegrationTests is specified //-->
+          <skipTests>true</skipTests>
+          <redirectTestOutputToFile>true</redirectTestOutputToFile>
+          <argLine>-Xmx3G -Djava.net.preferIPv4Stack=true -XX:MaxDirectMemorySize=2G -Dio.netty.leakDetection.level=PARANOID</argLine>
+          <forkMode>always</forkMode>
+          <forkedProcessTimeoutInSeconds>1800</forkedProcessTimeoutInSeconds>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <version>${apache-rat-plugin.version}</version>
+        <configuration>
+          <excludes>
+            <exclude>ChangeLog</exclude>
+            <exclude>CONFIG.ini</exclude>
+            <exclude>GROUPS</exclude>
+            <exclude>OWNERS</exclude>
+            <exclude>dist/**/*</exclude>
+            <exclude>docs/**/*</exclude>
+            <exclude>scripts/dev/reviewers</exclude>
+            <exclude>website/**/*</exclude>
+            <exclude>**/*.md</exclude>
+            <exclude>**/apidocs/*</exclude>
+            <exclude>**/dependency-reduced-pom.xml</exclude>
+            <exclude>**/org/apache/distributedlog/thrift/*</exclude>
+            <exclude>**/logs/*.log</exclude>
+            <exclude>**/target/**/*</exclude>
+            <!-- Git -->
+            <exclude>.git/**/*</exclude>
+            <exclude>.github/**/*</exclude>
+            <exclude>.gitignore</exclude>
+            <exclude>docker/.gitignore</exclude>
+            <exclude>.idea/**/*</exclude>
+            <!-- Intellij -->
+            <exclude>**/*.iml</exclude>
+            <exclude>**/*.iws</exclude>
+            <exclude>**/*.ipr</exclude>
+            <!-- SVN -->
+            <exclude>**/.svn/**/*</exclude>
+            <!-- Maven -->
+            <exclude>.repository/**</exclude>
+            <!-- Grafana -->
+            <exclude>docker/grafana/dashboards/*.json</exclude>
+          </excludes>
+          <consoleOutput>true</consoleOutput>
+        </configuration>
+      </plugin>
+      <!-- Report jacoco coverage to coveralls.io -->
+      <plugin>
+        <groupId>org.eluder.coveralls</groupId>
+        <artifactId>coveralls-maven-plugin</artifactId>
+        <version>${coveralls-maven-plugin.version}</version>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>cobertura-maven-plugin</artifactId>
+        <version>${cobertura-maven-plugin.version}</version>
+        <configuration>
+          <formats>
+            <format>html</format>
+            <format>xml</format>
+          </formats>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <profiles>
+    <profile>
+      <id>distributedlog</id>
+      <activation>
+        <property>
+          <name>distributedlog</name>
+        </property>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-surefire-plugin</artifactId>
+            <configuration>
+              <skipTests>false</skipTests>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>
diff --git a/stream/distributedlog/protocol/pom.xml b/stream/distributedlog/protocol/pom.xml
index bb355c6..9d3f770 100644
--- a/stream/distributedlog/protocol/pom.xml
+++ b/stream/distributedlog/protocol/pom.xml
@@ -20,10 +20,10 @@
   <parent>
     <groupId>org.apache.distributedlog</groupId>
     <artifactId>distributedlog</artifactId>
-    <version>0.6.0-SNAPSHOT</version>
+    <version>4.7.0-SNAPSHOT</version>
   </parent>
   <artifactId>distributedlog-protocol</artifactId>
-  <name>Apache DistributedLog :: Protocol</name>
+  <name>Apache BookKeeper :: DistributedLog :: Protocol</name>
   <dependencies>
     <dependency>
       <groupId>org.apache.distributedlog</groupId>
@@ -31,34 +31,10 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.projectlombok</groupId>
-      <artifactId>lombok</artifactId>
-      <version>${lombok.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
       <groupId>io.netty</groupId>
       <artifactId>netty-buffer</artifactId>
       <version>${netty.version}</version>
     </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <version>${junit.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-core</artifactId>
-      <version>${mockito.version}</version>
-      <scope>test</scope>
-    </dependency> 
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-log4j12</artifactId>
-      <version>${slf4j.version}</version>
-      <scope>test</scope>
-    </dependency>
   </dependencies>
   <build>
     <plugins>
@@ -75,8 +51,8 @@
         </executions>
       </plugin>
       <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>findbugs-maven-plugin</artifactId>
+        <groupId>com.github.spotbugs</groupId>
+        <artifactId>spotbugs-maven-plugin</artifactId>
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
@@ -89,13 +65,13 @@
             <version>${puppycrawl.checkstyle.version}</version>
           </dependency>
           <dependency>
-            <groupId>org.apache.distributedlog</groupId>
-            <artifactId>distributedlog-build-tools</artifactId>
-            <version>${project.version}</version>
+            <groupId>org.apache.bookkeeper</groupId>
+            <artifactId>buildtools</artifactId>
+            <version>${project.parent.version}</version>
           </dependency>
         </dependencies>
         <configuration>
-          <configLocation>distributedlog/checkstyle.xml</configLocation>
+          <configLocation>bookkeeper/checkstyle.xml</configLocation>
           <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
           <consoleOutput>true</consoleOutput>
           <failOnViolation>true</failOnViolation>

-- 
To stop receiving notification emails like this one, please contact
sijie@apache.org.