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/07 05:58:32 UTC

[bookkeeper] branch master updated: Issue 915: Part 2 - Add crc32c integer/long checksum utils under `com.scurrilous.circe.checksum` (#917)

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 3127848  Issue 915: Part 2 - Add crc32c integer/long checksum utils under `com.scurrilous.circe.checksum` (#917)
3127848 is described below

commit 312784812aa1349bf2864a5da07f6ace7ff687ba
Author: Sijie Guo <gu...@gmail.com>
AuthorDate: Sat Jan 6 21:58:30 2018 -0800

    Issue 915: Part 2 - Add crc32c integer/long checksum utils under `com.scurrilous.circe.checksum` (#917)
    
    This is the part 2 change of #915. It is based on #916, adding crc32c int/long checksum util classes under com.scurrilous.circe.checksum. It also fixes a incremental checksum issue on long crc32c. I added bunch of test cases to ensure the both long/int crc32 checksum is correct on incremental cases.
---
 .../main/resources/bookkeeper/findbugsExclude.xml  |   6 +
 circe-checksum/pom.xml                             |   6 +
 .../circe/checksum/Crc32cIntChecksum.java          |  85 +++++++++++++
 .../circe/checksum/Crc32cLongChecksum.java         |  91 ++++++++++++++
 .../circe/checksum/Crc32cSse42Provider.java        |  61 +++++++++
 .../com/scurrilous/circe/crc/ReflectedLongCrc.java |   6 +-
 .../scurrilous/circe/checksum/ChecksumTest.java    | 140 +++++++++++++++++++++
 .../java/com/scurrilous/circe/crc/CRCTest.java     |  25 +++-
 circe-checksum/src/test/resources/log4j.properties |  42 +++++++
 pom.xml                                            |   1 +
 10 files changed, 458 insertions(+), 5 deletions(-)

diff --git a/buildtools/src/main/resources/bookkeeper/findbugsExclude.xml b/buildtools/src/main/resources/bookkeeper/findbugsExclude.xml
index 4410382..0328f7d 100644
--- a/buildtools/src/main/resources/bookkeeper/findbugsExclude.xml
+++ b/buildtools/src/main/resources/bookkeeper/findbugsExclude.xml
@@ -16,6 +16,12 @@
    limitations under the License.
 //-->
 <FindBugsFilter>
+  <!-- circe-checksum -->
+  <Match>
+    <!-- imported code -->
+    <Class name="~com\.scurrilous\.circe.*" />
+  </Match>
+  <!-- bookkeeper-proto -->
   <Match>
     <!-- generated code, we can't be held responsible for findbugs in it //-->
     <Class name="~org\.apache\.bookkeeper\.proto\.DataFormats.*" />
diff --git a/circe-checksum/pom.xml b/circe-checksum/pom.xml
index 001fd7c..ddcd0f2 100644
--- a/circe-checksum/pom.xml
+++ b/circe-checksum/pom.xml
@@ -49,6 +49,12 @@
     </dependency>
 
     <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-buffer</artifactId>
+      <version>${netty.version}</version>
+    </dependency>
+
+    <dependency>
       <groupId>org.testng</groupId>
       <artifactId>testng</artifactId>
       <version>6.8.5</version>
diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cIntChecksum.java b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cIntChecksum.java
new file mode 100644
index 0000000..cdf3d8d
--- /dev/null
+++ b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cIntChecksum.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package com.scurrilous.circe.checksum;
+
+import static com.scurrilous.circe.params.CrcParameters.CRC32C;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.scurrilous.circe.IncrementalIntHash;
+import com.scurrilous.circe.crc.Sse42Crc32C;
+import com.scurrilous.circe.crc.StandardCrcProvider;
+import io.netty.buffer.ByteBuf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Crc32cIntChecksum {
+
+    private static final Logger log = LoggerFactory.getLogger(Crc32cIntChecksum.class);
+
+    @VisibleForTesting
+    static IncrementalIntHash CRC32C_HASH;
+
+    static {
+        if (Sse42Crc32C.isSupported()) {
+            CRC32C_HASH = new Crc32cSse42Provider().getIncrementalInt(CRC32C);
+            log.info("SSE4.2 CRC32C provider initialized");
+        } else {
+            CRC32C_HASH = new StandardCrcProvider().getIncrementalInt(CRC32C);
+        }
+    }
+
+    /**
+     * Computes crc32c checksum: if it is able to load crc32c native library then it computes using that native library
+     * which is faster as it computes using hardware machine instruction else it computes using crc32c algo.
+     *
+     * @param payload
+     * @return
+     */
+    public static int computeChecksum(ByteBuf payload) {
+        if (payload.hasMemoryAddress() && (CRC32C_HASH instanceof Sse42Crc32C)) {
+            return CRC32C_HASH.calculate(payload.memoryAddress() + payload.readerIndex(), payload.readableBytes());
+        } else if (payload.hasArray()) {
+            return CRC32C_HASH.calculate(payload.array(), payload.arrayOffset() + payload.readerIndex(),
+                payload.readableBytes());
+        } else {
+            return CRC32C_HASH.calculate(payload.nioBuffer());
+        }
+    }
+
+
+    /**
+     * Computes incremental checksum with input previousChecksum and input payload
+     *
+     * @param previousChecksum : previously computed checksum
+     * @param payload
+     * @return
+     */
+    public static int resumeChecksum(int previousChecksum, ByteBuf payload) {
+        if (payload.hasMemoryAddress() && (CRC32C_HASH instanceof Sse42Crc32C)) {
+            return CRC32C_HASH.resume(previousChecksum, payload.memoryAddress() + payload.readerIndex(),
+                payload.readableBytes());
+        } else if (payload.hasArray()) {
+            return CRC32C_HASH.resume(previousChecksum, payload.array(), payload.arrayOffset() + payload.readerIndex(),
+                payload.readableBytes());
+        } else {
+            return CRC32C_HASH.resume(previousChecksum, payload.nioBuffer());
+        }
+    }
+
+}
diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cLongChecksum.java b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cLongChecksum.java
new file mode 100644
index 0000000..401b073
--- /dev/null
+++ b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cLongChecksum.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package com.scurrilous.circe.checksum;
+
+import static com.scurrilous.circe.params.CrcParameters.CRC32C;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.scurrilous.circe.IncrementalIntHash;
+import com.scurrilous.circe.crc.Sse42Crc32C;
+import com.scurrilous.circe.crc.StandardCrcProvider;
+import io.netty.buffer.ByteBuf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Crc32cLongChecksum {
+
+    private static final Logger log = LoggerFactory.getLogger(Crc32cLongChecksum.class);
+
+    @VisibleForTesting
+    static IncrementalIntHash CRC32C_HASH;
+
+    static {
+        if (Sse42Crc32C.isSupported()) {
+            CRC32C_HASH = new Crc32cSse42Provider().getIncrementalInt(CRC32C);
+            if (log.isDebugEnabled()) {
+                log.debug("SSE4.2 CRC32C provider initialized");
+            }
+        } else {
+            CRC32C_HASH = new StandardCrcProvider().getIncrementalInt(CRC32C);
+        }
+    }
+
+    /**
+     * Computes crc32c checksum: if it is able to load crc32c native library then it computes using that native library
+     * which is faster as it computes using hardware machine instruction else it computes using crc32c algo.
+     *
+     * @param payload
+     * @return
+     */
+    public static long computeChecksum(ByteBuf payload) {
+        int crc;
+        if (payload.hasMemoryAddress() && (CRC32C_HASH instanceof Sse42Crc32C)) {
+            crc = CRC32C_HASH.calculate(payload.memoryAddress() + payload.readerIndex(), payload.readableBytes());
+        } else if (payload.hasArray()) {
+            crc = CRC32C_HASH.calculate(payload.array(), payload.arrayOffset() + payload.readerIndex(),
+                payload.readableBytes());
+        } else {
+            crc = CRC32C_HASH.calculate(payload.nioBuffer());
+        }
+        return crc & 0xffffffffL;
+    }
+
+
+    /**
+     * Computes incremental checksum with input previousChecksum and input payload
+     *
+     * @param previousChecksum : previously computed checksum
+     * @param payload
+     * @return
+     */
+    public static long resumeChecksum(long previousChecksum, ByteBuf payload) {
+        int crc = (int) previousChecksum;
+        if (payload.hasMemoryAddress() && (CRC32C_HASH instanceof Sse42Crc32C)) {
+            crc = CRC32C_HASH.resume(crc, payload.memoryAddress() + payload.readerIndex(),
+                payload.readableBytes());
+        } else if (payload.hasArray()) {
+            crc = CRC32C_HASH.resume(crc, payload.array(), payload.arrayOffset() + payload.readerIndex(),
+                payload.readableBytes());
+        } else {
+            crc = CRC32C_HASH.resume(crc, payload.nioBuffer());
+        }
+        return crc & 0xffffffffL;
+    }
+
+}
diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cSse42Provider.java b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cSse42Provider.java
new file mode 100644
index 0000000..65fd37b
--- /dev/null
+++ b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cSse42Provider.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package com.scurrilous.circe.checksum;
+
+import com.scurrilous.circe.Hash;
+import com.scurrilous.circe.HashParameters;
+import com.scurrilous.circe.HashSupport;
+import com.scurrilous.circe.StatelessHash;
+import com.scurrilous.circe.crc.Sse42Crc32C;
+import com.scurrilous.circe.impl.AbstractHashProvider;
+import com.scurrilous.circe.params.CrcParameters;
+import java.util.EnumSet;
+
+public final class Crc32cSse42Provider extends AbstractHashProvider<HashParameters> {
+
+    // Default chunks of 32 KB, then 4 KB, then 512 bytes
+    private static final int[] DEFAULT_CHUNK = new int[] { 4096, 512, 64 };
+
+    public Crc32cSse42Provider() {
+        super(HashParameters.class);
+    }
+
+    @Override
+    protected EnumSet<HashSupport> querySupportTyped(HashParameters params) {
+        if (isCrc32C(params) && Sse42Crc32C.isSupported())
+            return EnumSet.allOf(HashSupport.class);
+        return EnumSet.noneOf(HashSupport.class);
+    }
+
+    @Override
+    protected Hash get(HashParameters params, EnumSet<HashSupport> required) {
+        if (isCrc32C(params) && Sse42Crc32C.isSupported())
+            return getCacheable(params, required);
+        throw new UnsupportedOperationException();
+    }
+
+    private static boolean isCrc32C(HashParameters params) {
+        return params.equals(CrcParameters.CRC32C);
+    }
+
+    @Override
+    protected StatelessHash createCacheable(HashParameters params, EnumSet<HashSupport> required) {
+        return new Sse42Crc32C(DEFAULT_CHUNK);
+    }
+}
diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/crc/ReflectedLongCrc.java b/circe-checksum/src/main/java/com/scurrilous/circe/crc/ReflectedLongCrc.java
index 951fbec..ec07cab 100644
--- a/circe-checksum/src/main/java/com/scurrilous/circe/crc/ReflectedLongCrc.java
+++ b/circe-checksum/src/main/java/com/scurrilous/circe/crc/ReflectedLongCrc.java
@@ -36,8 +36,12 @@ final class ReflectedLongCrc extends AbstractLongCrc {
     }
 
     @Override
+    protected long initial() {
+        return reflect(super.initial());
+    }
+
+    @Override
     protected long resumeRaw(long crc, byte[] input, int index, int length) {
-        crc = reflect(crc);
         for (int i = 0; i < length; ++i)
             crc = table[(int) (crc ^ input[index + i]) & 0xff] ^ (crc >>> 8);
         return crc;
diff --git a/circe-checksum/src/test/java/com/scurrilous/circe/checksum/ChecksumTest.java b/circe-checksum/src/test/java/com/scurrilous/circe/checksum/ChecksumTest.java
new file mode 100644
index 0000000..f6b0b9b
--- /dev/null
+++ b/circe-checksum/src/test/java/com/scurrilous/circe/checksum/ChecksumTest.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package com.scurrilous.circe.checksum;
+
+import static com.scurrilous.circe.params.CrcParameters.CRC32C;
+import static org.testng.Assert.assertEquals;
+
+import com.scurrilous.circe.IncrementalIntHash;
+import com.scurrilous.circe.IncrementalLongHash;
+import com.scurrilous.circe.crc.StandardCrcProvider;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.testng.annotations.Test;
+
+/**
+ * Verify circe checksum.
+ */
+public class ChecksumTest {
+
+    @Test
+    public void testCrc32cValue() {
+        final byte[] bytes = "Some String".getBytes();
+        int checksum = Crc32cIntChecksum.computeChecksum(Unpooled.wrappedBuffer(bytes));
+
+        assertEquals(608512271, checksum);
+    }
+
+    @Test
+    public void testCrc32cValueResume() {
+        final byte[] bytes = "Some String".getBytes();
+        int checksum = Crc32cIntChecksum.resumeChecksum(0, Unpooled.wrappedBuffer(bytes));
+
+        assertEquals(608512271, checksum);
+    }
+
+    @Test
+    public void testCrc32cValueIncremental() {
+        final byte[] bytes = "Some String".getBytes();
+
+        int checksum = Crc32cIntChecksum.CRC32C_HASH.calculate(bytes, 0, bytes.length);
+        assertEquals(608512271, checksum);
+
+        checksum = Crc32cIntChecksum.CRC32C_HASH.calculate(bytes, 0, 1);
+        for (int i = 1; i < bytes.length; i++) {
+            checksum = Crc32cIntChecksum.CRC32C_HASH.resume(checksum, bytes, i, 1);
+        }
+        assertEquals(608512271, checksum);
+
+        checksum = Crc32cIntChecksum.CRC32C_HASH.calculate(bytes, 0, 4);
+        checksum = Crc32cIntChecksum.CRC32C_HASH.resume(checksum, bytes, 4, 7);
+        assertEquals(608512271, checksum);
+
+
+        ByteBuf buffer = Unpooled.wrappedBuffer(bytes, 0, 4);
+        checksum = Crc32cIntChecksum.computeChecksum(buffer);
+        checksum = Crc32cIntChecksum.resumeChecksum(
+            checksum, Unpooled.wrappedBuffer(bytes, 4, bytes.length - 4));
+
+        assertEquals(608512271, checksum);
+    }
+
+    @Test
+    public void testCrc32cLongValue() {
+        final byte[] bytes = "Some String".getBytes();
+        long checksum = Crc32cIntChecksum.computeChecksum(Unpooled.wrappedBuffer(bytes));
+
+        assertEquals(608512271L, checksum);
+    }
+
+    @Test
+    public void testCrc32cLongValueResume() {
+        final byte[] bytes = "Some String".getBytes();
+        long checksum = Crc32cIntChecksum.resumeChecksum(0, Unpooled.wrappedBuffer(bytes));
+
+        assertEquals(608512271L, checksum);
+    }
+
+    @Test
+    public void testCrc32cLongValueIncremental() {
+        final byte[] bytes = "Some String".getBytes();
+
+        long checksum = Crc32cLongChecksum.computeChecksum(Unpooled.wrappedBuffer(bytes));
+        assertEquals(608512271, checksum);
+
+        checksum = Crc32cLongChecksum.computeChecksum(Unpooled.wrappedBuffer(bytes, 0, 1));
+        for (int i = 1; i < bytes.length; i++) {
+            checksum = Crc32cLongChecksum.resumeChecksum(
+                checksum, Unpooled.wrappedBuffer(bytes, i, 1));
+        }
+        assertEquals(608512271, checksum);
+
+        ByteBuf buffer = Unpooled.wrappedBuffer(bytes, 0, 4);
+        checksum = Crc32cLongChecksum.computeChecksum(buffer);
+        checksum = Crc32cLongChecksum.resumeChecksum(
+            checksum, Unpooled.wrappedBuffer(bytes, 4, bytes.length - 4));
+
+        assertEquals(608512271, checksum);
+    }
+
+    @Test
+    public void testCRC32CIncrementalLong() {
+        IncrementalLongHash crc32c = new StandardCrcProvider().getIncrementalLong(CRC32C);
+        String data = "data";
+        String combine = data + data;
+
+        long combineChecksum = crc32c.calculate(combine.getBytes());
+        long dataChecksum = crc32c.calculate(data.getBytes());
+        long incrementalChecksum = crc32c.resume(dataChecksum, data.getBytes());
+        assertEquals(combineChecksum, incrementalChecksum);
+    }
+
+    @Test
+    public void testCRC32CIncrementalInt() {
+        IncrementalIntHash crc32c = new StandardCrcProvider().getIncrementalInt(CRC32C);
+        String data = "data";
+        String combine = data + data;
+
+        int combineChecksum = crc32c.calculate(combine.getBytes());
+        int dataChecksum = crc32c.calculate(data.getBytes());
+        int incrementalChecksum = crc32c.resume(dataChecksum, data.getBytes());
+        assertEquals(combineChecksum, incrementalChecksum);
+    }
+
+}
diff --git a/circe-checksum/src/test/java/com/scurrilous/circe/crc/CRCTest.java b/circe-checksum/src/test/java/com/scurrilous/circe/crc/CRCTest.java
index c2803c9..e5eb77d 100644
--- a/circe-checksum/src/test/java/com/scurrilous/circe/crc/CRCTest.java
+++ b/circe-checksum/src/test/java/com/scurrilous/circe/crc/CRCTest.java
@@ -26,6 +26,7 @@ import static com.scurrilous.circe.params.CrcParameters.CRC64;
 import static com.scurrilous.circe.params.CrcParameters.CRC64_XZ;
 import static org.testng.Assert.assertEquals;
 
+import com.scurrilous.circe.IncrementalLongHash;
 import java.nio.charset.Charset;
 
 import org.testng.annotations.Test;
@@ -163,12 +164,12 @@ public class CRCTest {
     }
     
     @Test
-    public void testCRC32CIncremental() {
+    public void testCRC32CIncrementalInt() {
         // reflected
-        testIncremental(PROVIDER.getIncrementalInt(CRC32C));
+        testIncrementalInt(PROVIDER.getIncrementalInt(CRC32C));
     }
 
-    private void testIncremental(IncrementalIntHash hash) {
+    private void testIncrementalInt(IncrementalIntHash hash) {
         final String data = "data";
         final String combined = data + data;
 
@@ -177,4 +178,20 @@ public class CRCTest {
         final int incrementalChecksum = hash.resume(dataChecksum, data.getBytes(ASCII));
         assertEquals(combinedChecksum, incrementalChecksum);
     }
-}
\ No newline at end of file
+
+    @Test
+    public void testCRC32CIncrementalLong() {
+        // reflected
+        testIncrementalLong(PROVIDER.getIncrementalLong(CRC32C));
+    }
+
+    private void testIncrementalLong(IncrementalLongHash hash) {
+        final String data = "data";
+        final String combined = data + data;
+
+        final long dataChecksum = hash.calculate(data.getBytes(ASCII));
+        final long combinedChecksum = hash.calculate(combined.getBytes(ASCII));
+        final long incrementalChecksum = hash.resume(dataChecksum, data.getBytes(ASCII));
+        assertEquals(combinedChecksum, incrementalChecksum);
+    }
+}
diff --git a/circe-checksum/src/test/resources/log4j.properties b/circe-checksum/src/test/resources/log4j.properties
new file mode 100644
index 0000000..10ae6bf
--- /dev/null
+++ b/circe-checksum/src/test/resources/log4j.properties
@@ -0,0 +1,42 @@
+#
+#
+# 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.
+#
+#
+
+#
+# Bookkeeper Logging Configuration
+#
+
+# Format is "<default threshold> (, <appender>)+
+
+# DEFAULT: console appender only, level INFO
+bookkeeper.root.logger=INFO,CONSOLE
+log4j.rootLogger=${bookkeeper.root.logger}
+
+#
+# Log INFO level and above messages to the console
+#
+log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
+log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+#disable zookeeper logging
+log4j.logger.org.apache.zookeeper=OFF
+log4j.logger.org.apache.bookkeeper.bookie=INFO
+log4j.logger.org.apache.bookkeeper.meta=INFO
diff --git a/pom.xml b/pom.xml
index d284723..11c798c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -318,6 +318,7 @@
             <exclude>**/*.iml</exclude>
             <exclude>site/**</exclude>
             <exclude>.repository/**</exclude>
+            <exclude>**/META-INF/**</exclude>
           </excludes>
           <consoleOutput>true</consoleOutput>
         </configuration>

-- 
To stop receiving notification emails like this one, please contact
['"commits@bookkeeper.apache.org" <co...@bookkeeper.apache.org>'].