You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2017/11/12 07:02:45 UTC

[GitHub] jiazhai closed pull request #718: Issue 490: Add a flag to not serialize `ctime` field

jiazhai closed pull request #718: Issue 490: Add a flag to not serialize `ctime` field
URL: https://github.com/apache/bookkeeper/pull/718
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java
index 4cc62a06e..fc37c04cf 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java
@@ -93,7 +93,14 @@
     LedgerCreateOp(BookKeeper bk, int ensembleSize, int writeQuorumSize, int ackQuorumSize, DigestType digestType,
             byte[] passwd, CreateCallback cb, Object ctx, final Map<String, byte[]> customMetadata) {
         this.bk = bk;
-        this.metadata = new LedgerMetadata(ensembleSize, writeQuorumSize, ackQuorumSize, digestType, passwd, customMetadata);
+        this.metadata = new LedgerMetadata(
+            ensembleSize,
+            writeQuorumSize,
+            ackQuorumSize,
+            digestType,
+            passwd,
+            customMetadata,
+            bk.getConf().getStoreSystemtimeAsLedgerCreationTime());
         this.digestType = digestType;
         this.passwd = passwd;
         this.cb = cb;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
index 5683105de..0a0b87c42 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
@@ -75,6 +75,7 @@
     private long length;
     private long lastEntryId;
     private long ctime;
+    private boolean storeSystemtimeAsLedgerCreationTime;
 
     private LedgerMetadataFormat.State state;
     private SortedMap<Long, ArrayList<BookieSocketAddress>> ensembles =
@@ -88,12 +89,18 @@
 
     private Map<String, byte[]> customMetadata = Maps.newHashMap();
 
-    public LedgerMetadata(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
-                          BookKeeper.DigestType digestType, byte[] password, Map<String, byte[]> customMetadata) {
+    public LedgerMetadata(int ensembleSize,
+                          int writeQuorumSize,
+                          int ackQuorumSize,
+                          BookKeeper.DigestType digestType,
+                          byte[] password,
+                          Map<String, byte[]> customMetadata,
+                          boolean storeSystemtimeAsLedgerCreationTime) {
         this.ensembleSize = ensembleSize;
         this.writeQuorumSize = writeQuorumSize;
         this.ackQuorumSize = ackQuorumSize;
         this.ctime = System.currentTimeMillis();
+        this.storeSystemtimeAsLedgerCreationTime = storeSystemtimeAsLedgerCreationTime;
 
         /*
          * It is set in PendingReadOp.readEntryComplete, and
@@ -115,7 +122,7 @@ public LedgerMetadata(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
 
     public LedgerMetadata(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
             BookKeeper.DigestType digestType, byte[] password) {
-        this(ensembleSize, writeQuorumSize, ackQuorumSize, digestType, password, null);
+        this(ensembleSize, writeQuorumSize, ackQuorumSize, digestType, password, null, false);
     }
 
     /**
@@ -282,19 +289,15 @@ void setCustomMetadata(Map<String, byte[]> customMetadata) {
         this.customMetadata = customMetadata;
     }
 
-    /**
-     * Generates a byte array of this object
-     *
-     * @return the metadata serialized into a byte array
-     */
-    public byte[] serialize() {
-        if (metadataFormatVersion == 1) {
-            return serializeVersion1();
-        }
+    LedgerMetadataFormat buildProtoFormat() {
         LedgerMetadataFormat.Builder builder = LedgerMetadataFormat.newBuilder();
         builder.setQuorumSize(writeQuorumSize).setAckQuorumSize(ackQuorumSize)
             .setEnsembleSize(ensembleSize).setLength(length)
-            .setState(state).setLastEntryId(lastEntryId).setCtime(ctime);
+            .setState(state).setLastEntryId(lastEntryId);
+
+        if (storeSystemtimeAsLedgerCreationTime) {
+            builder.setCtime(ctime);
+        }
 
         if (hasPassword) {
             builder.setDigestType(digestType).setPassword(ByteString.copyFrom(password));
@@ -316,10 +319,22 @@ void setCustomMetadata(Map<String, byte[]> customMetadata) {
             }
             builder.addSegment(segmentBuilder.build());
         }
+        return builder.build();
+    }
+
+    /**
+     * Generates a byte array of this object
+     *
+     * @return the metadata serialized into a byte array
+     */
+    public byte[] serialize() {
+        if (metadataFormatVersion == 1) {
+            return serializeVersion1();
+        }
 
         StringBuilder s = new StringBuilder();
         s.append(VERSION_KEY).append(tSplitter).append(CURRENT_METADATA_FORMAT_VERSION).append(lSplitter);
-        s.append(TextFormat.printToString(builder.build()));
+        s.append(TextFormat.printToString(buildProtoFormat()));
         if (LOG.isDebugEnabled()) {
             LOG.debug("Serialized config: {}", s);
         }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java
index 8725a6ce6..f4ffd521f 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java
@@ -114,6 +114,9 @@
     protected final static String ENSEMBLE_PLACEMENT_POLICY = "ensemblePlacementPolicy";
     protected final static String NETWORK_TOPOLOGY_STABILIZE_PERIOD_SECONDS = "networkTopologyStabilizePeriodSeconds";
 
+    // Ledger Metadata Parameters
+    protected static final String STORE_SYSTEMTIME_AS_LEDGER_CREATION_TIME = "storeSystemTimeAsLedgerCreationTime";
+
     // Stats
     protected final static String ENABLE_TASK_EXECUTION_STATS = "enableTaskExecutionStats";
     protected final static String TASK_EXECUTION_WARN_TIME_MICROS = "taskExecutionWarnTimeMicros";
@@ -1632,9 +1635,10 @@ public ClientConfiguration setNettyUsePooledBuffers(boolean enabled) {
      * @param regClientClass
      *            ClientClass
      */
-    public void setRegistrationClientClass(
+    public ClientConfiguration setRegistrationClientClass(
             Class<? extends RegistrationClient> regClientClass) {
         setProperty(REGISTRATION_CLIENT_CLASS, regClientClass);
+        return this;
     }
 
     /**
@@ -1648,4 +1652,29 @@ public void setRegistrationClientClass(
                 ZKRegistrationClient.class, RegistrationClient.class,
                 defaultLoader);
     }
+
+    /**
+     * Enable the client to use system time as the ledger creation time.
+     *
+     * <p>If this is enabled, the client will write a ctime field into the ledger metadata.
+     * Otherwise, nothing will be written. The creation time of this ledger will be the ctime
+     * of the metadata record in metadata store.
+     *
+     * @param enabled flag to enable/disable client using system time as the ledger creation time.
+     */
+    public ClientConfiguration setStoreSystemtimeAsLedgerCreationTime(boolean enabled) {
+        setProperty(STORE_SYSTEMTIME_AS_LEDGER_CREATION_TIME, enabled);
+        return this;
+    }
+
+    /**
+     * Return the flag that indicates whether client is using system time as the ledger creation time when
+     * creating ledgers.
+     *
+     * @return the flag that indicates whether client is using system time as the ledger creation time when
+     *         creating ledgers.
+     */
+    public boolean getStoreSystemtimeAsLedgerCreationTime() {
+        return getBoolean(STORE_SYSTEMTIME_AS_LEDGER_CREATION_TIME, false);
+    }
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java
new file mode 100644
index 000000000..5cb14686e
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java
@@ -0,0 +1,70 @@
+/*
+ * 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 org.apache.bookkeeper.client;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collections;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat;
+import org.junit.Test;
+
+/**
+ * Unit test for ledger metadata
+ */
+public class LedgerMetadataTest {
+
+    @Test
+    public void testStoreSystemtimeAsLedgerCtimeEnabled()
+            throws Exception {
+        byte[] passwd = "testPasswd".getBytes(UTF_8);
+
+        LedgerMetadata lm = new LedgerMetadata(
+            3,
+            3,
+            2,
+            DigestType.CRC32,
+            passwd,
+            Collections.emptyMap(),
+            true);
+        LedgerMetadataFormat format = lm.buildProtoFormat();
+        assertTrue(format.hasCtime());
+    }
+
+    @Test
+    public void testStoreSystemtimeAsLedgerCtimeDisabled()
+            throws Exception {
+        byte[] passwd = "testPasswd".getBytes(UTF_8);
+
+        LedgerMetadata lm = new LedgerMetadata(
+            3,
+            3,
+            2,
+            DigestType.CRC32,
+            passwd,
+            Collections.emptyMap(),
+            false);
+        LedgerMetadataFormat format = lm.buildProtoFormat();
+        assertFalse(format.hasCtime());
+    }
+
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ListLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ListLedgersTest.java
index 12d108a77..fa32ef653 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ListLedgersTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ListLedgersTest.java
@@ -101,28 +101,6 @@ public void testRemoveNotSupported()
 
     }
     
-    @Test
-    public void testCtimeRecorded()
-            throws Exception {
-
-        ClientConfiguration conf = new ClientConfiguration()
-                .setZkServers(zkUtil.getZooKeeperConnectString());
 
-        BookKeeper bkc = new BookKeeper(conf);
-
-        bkc.createLedger(digestType, "testPasswd".
-                getBytes()).close();
-
-        BookKeeperAdmin admin = new BookKeeperAdmin(zkUtil.
-                getZooKeeperConnectString());
-        Iterable<Long> iterable = admin.listLedgers();
-
-        for (Long lId : iterable) {
-            LedgerHandle ledger = bkc.openLedger(lId, digestType, "testPasswd".getBytes());
-            LedgerMetadata metaData = ledger.getLedgerMetadata();
-            Assert.assertTrue("ctime was not recorded", metaData.getCtime() > 0);
-        }
-
-    }
     
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java
index 946ee4e1e..edda096f8 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java
@@ -309,8 +309,14 @@ public void testDeleteLedgerBookKeeperClosed() throws Exception {
     protected LedgerMetadata generateLedgerMetadata(int ensembleSize,
         int writeQuorumSize, int ackQuorumSize, byte[] password,
         Map<String, byte[]> customMetadata) {
-        LedgerMetadata ledgerMetadata = new LedgerMetadata(ensembleSize, writeQuorumSize,
-            ackQuorumSize, BookKeeper.DigestType.CRC32, password, customMetadata);
+        LedgerMetadata ledgerMetadata = new LedgerMetadata(
+            ensembleSize,
+            writeQuorumSize,
+            ackQuorumSize,
+            BookKeeper.DigestType.CRC32,
+            password,
+            customMetadata,
+            true);
         ledgerMetadata.addEnsemble(0, generateNewEnsemble(ensembleSize));
         return ledgerMetadata;
     }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java
index 6627f8287..7968850c9 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java
@@ -524,7 +524,7 @@ public void testTriggerAuditorWithNoPendingAuditTask() throws Exception {
         int numofledgers = 5;
         Random rand = new Random();
         for (int i = 0; i < numofledgers; i++) {
-            LedgerMetadata metadata = new LedgerMetadata(3, 2, 2, DigestType.CRC32, "passwd".getBytes(), null);
+            LedgerMetadata metadata = new LedgerMetadata(3, 2, 2, DigestType.CRC32, "passwd".getBytes());
             ArrayList<BookieSocketAddress> ensemble = new ArrayList<BookieSocketAddress>();
             ensemble.add(new BookieSocketAddress("99.99.99.99:9999"));
             ensemble.add(new BookieSocketAddress("11.11.11.11:1111"));


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services