You are viewing a plain text version of this content. The canonical link for it is here.
Posted to distributedlog-issues@bookkeeper.apache.org by gi...@git.apache.org on 2017/10/13 02:48:32 UTC

[GitHub] jiazhai commented on a change in pull request #210: Issue 209: Support rename log

jiazhai commented on a change in pull request #210: Issue 209: Support rename log
URL: https://github.com/apache/distributedlog/pull/210#discussion_r144455505
 
 

 ##########
 File path: distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
 ##########
 @@ -620,4 +631,271 @@ public void processResult(int rc, String path, Object ctx) {
         }
         return promise;
     }
+
+    //
+    // Rename Log
+    //
+
+    @Override
+    public CompletableFuture<Void> renameLog(URI uri, String oldStreamName, String newStreamName) {
+        return getLog(
+            uri,
+            oldStreamName,
+            true,
+            false
+        ).thenCompose(metadata -> renameLogMetadata(uri, metadata, newStreamName));
+    }
+
+    private CompletableFuture<Void> renameLogMetadata(URI uri,
+                                                      LogMetadataForWriter oldMetadata,
+                                                      String newStreamName) {
+
+
+        final LinkedList<Op> createOps = Lists.newLinkedList();
+        final LinkedList<Op> deleteOps = Lists.newLinkedList();
+
+        List<ACL> acls = zooKeeperClient.getDefaultACL();
+
+        // get the root path
+        String oldRootPath = oldMetadata.getLogRootPath();
+        String newRootPath = LogMetadata.getLogRootPath(
+            uri, newStreamName, conf.getUnpartitionedStreamName());
+
+        // 0. the log path
+        deleteOps.addFirst(Op.delete(
+            LogMetadata.getLogStreamPath(uri, oldMetadata.getLogName()), -1));
+
+        // 1. the root path
+        createOps.addLast(Op.create(
+            newRootPath, EMPTY_BYTES, acls, CreateMode.PERSISTENT));
+        deleteOps.addFirst(Op.delete(
+            oldRootPath, -1));
+
+        // 2. max id
+        Versioned<byte[]> maxTxIdData = oldMetadata.getMaxTxIdData();
+        deleteOldPathAndCreateNewPath(
+            oldRootPath, MAX_TXID_PATH, maxTxIdData,
+            newRootPath, DLUtils.serializeTransactionId(0L), acls,
+            createOps, deleteOps
+        );
+
+        // 3. version
+        createOps.addLast(Op.create(
+            newRootPath + VERSION_PATH, intToBytes(LAYOUT_VERSION), acls, CreateMode.PERSISTENT));
+        deleteOps.addFirst(Op.delete(
+            oldRootPath + VERSION_PATH, -1));
+
+        // 4. lock path (NOTE: if the stream is locked by a writer, then the delete will fail as you can not
 
 Review comment:
   The rename operation will fail if the stream is still locked, right?  
   If it is not complex, how about also add a test cases for this situation?
 
----------------------------------------------------------------
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