You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2020/06/25 07:50:30 UTC

[GitHub] [pulsar] zhanghaou opened a new pull request #7357: Fix bug related to managedLedger properties

zhanghaou opened a new pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357


   Fix issue #7161
   
   related to [https://github.com/apache/pulsar/pull/7222](https://github.com/apache/pulsar/pull/7222)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] zhanghaou commented on a change in pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
zhanghaou commented on a change in pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#discussion_r446590675



##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
##########
@@ -3271,57 +3275,89 @@ public long getOffloadedSize() {
     }
 
     @Override
-    public void setProperties(Map<String, String> properties) throws InterruptedException {
+    public void setProperty(String key, String value) throws InterruptedException, ManagedLedgerException {
+        Map<String, String> map = new HashMap<>();
+        map.put(key, value);
+        updateProperties(map, false, null);
+    }
+
+    @Override
+    public void asyncSetProperty(String key, String value, final UpdatePropertiesCallback callback, Object ctx) {
+        Map<String, String> map = new HashMap<>();
+        map.put(key, value);
+        asyncUpdateProperties(map, false, null, callback, ctx);
+    }
+
+    @Override
+    public void deleteProperty(String key) throws InterruptedException, ManagedLedgerException {
+        updateProperties(null, true, key);
+    }
+
+    @Override
+    public void asyncDeleteProperty(String key, final UpdatePropertiesCallback callback, Object ctx) {
+        asyncUpdateProperties(null, true, key, callback, ctx);
+    }
+
+    @Override
+    public void setProperties(Map<String, String> properties) throws InterruptedException, ManagedLedgerException {
+        updateProperties(properties, false, null);
+    }
+
+    @Override
+    public void asyncSetProperties(Map<String, String> properties, final UpdatePropertiesCallback callback,
+        Object ctx) {
+        asyncUpdateProperties(properties, false, null, callback, ctx);
+    }
+
+    private void updateProperties(Map<String, String> properties, boolean isDelete,
+        String deleteKey) throws InterruptedException, ManagedLedgerException {
         final CountDownLatch latch = new CountDownLatch(1);
-        this.asyncSetProperties(properties, new SetPropertiesCallback() {
+        AtomicBoolean isSucceed = new AtomicBoolean(false);
+        this.asyncUpdateProperties(properties, isDelete, deleteKey, new UpdatePropertiesCallback() {
             @Override
-            public void setPropertiesComplete(Map<String, String> properties, Object ctx) {
+            public void updatePropertiesComplete(Map<String, String> properties, Object ctx) {
+                isSucceed.set(true);
                 latch.countDown();
             }
 
             @Override
-            public void setPropertiesFailed(ManagedLedgerException exception, Object ctx) {
+            public void updatePropertiesFailed(ManagedLedgerException exception, Object ctx) {
                 log.error("[{}] Update manageLedger's info failed:{}", name, exception.getMessage());
                 latch.countDown();
             }
         }, null);
 
-        latch.await();
+        if (!latch.await(AsyncOperationTimeoutSeconds, TimeUnit.SECONDS) || !isSucceed.get()) {
+            throw new ManagedLedgerException("Update properties failed");

Review comment:
       Done. Thanks for your guidance :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] merlimat commented on a change in pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
merlimat commented on a change in pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#discussion_r446548913



##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
##########
@@ -3271,57 +3275,89 @@ public long getOffloadedSize() {
     }
 
     @Override
-    public void setProperties(Map<String, String> properties) throws InterruptedException {
+    public void setProperty(String key, String value) throws InterruptedException, ManagedLedgerException {
+        Map<String, String> map = new HashMap<>();
+        map.put(key, value);
+        updateProperties(map, false, null);
+    }
+
+    @Override
+    public void asyncSetProperty(String key, String value, final UpdatePropertiesCallback callback, Object ctx) {
+        Map<String, String> map = new HashMap<>();
+        map.put(key, value);
+        asyncUpdateProperties(map, false, null, callback, ctx);
+    }
+
+    @Override
+    public void deleteProperty(String key) throws InterruptedException, ManagedLedgerException {
+        updateProperties(null, true, key);
+    }
+
+    @Override
+    public void asyncDeleteProperty(String key, final UpdatePropertiesCallback callback, Object ctx) {
+        asyncUpdateProperties(null, true, key, callback, ctx);
+    }
+
+    @Override
+    public void setProperties(Map<String, String> properties) throws InterruptedException, ManagedLedgerException {
+        updateProperties(properties, false, null);
+    }
+
+    @Override
+    public void asyncSetProperties(Map<String, String> properties, final UpdatePropertiesCallback callback,
+        Object ctx) {
+        asyncUpdateProperties(properties, false, null, callback, ctx);
+    }
+
+    private void updateProperties(Map<String, String> properties, boolean isDelete,
+        String deleteKey) throws InterruptedException, ManagedLedgerException {
         final CountDownLatch latch = new CountDownLatch(1);
-        this.asyncSetProperties(properties, new SetPropertiesCallback() {
+        AtomicBoolean isSucceed = new AtomicBoolean(false);
+        this.asyncUpdateProperties(properties, isDelete, deleteKey, new UpdatePropertiesCallback() {
             @Override
-            public void setPropertiesComplete(Map<String, String> properties, Object ctx) {
+            public void updatePropertiesComplete(Map<String, String> properties, Object ctx) {
+                isSucceed.set(true);
                 latch.countDown();
             }
 
             @Override
-            public void setPropertiesFailed(ManagedLedgerException exception, Object ctx) {
+            public void updatePropertiesFailed(ManagedLedgerException exception, Object ctx) {
                 log.error("[{}] Update manageLedger's info failed:{}", name, exception.getMessage());
                 latch.countDown();
             }
         }, null);
 
-        latch.await();
+        if (!latch.await(AsyncOperationTimeoutSeconds, TimeUnit.SECONDS) || !isSucceed.get()) {
+            throw new ManagedLedgerException("Update properties failed");

Review comment:
       We should use the exception returned int the `updatePropertiesFailed()` and propagate that back to the user. Take a look at https://github.com/apache/pulsar/blob/master/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java#L1096




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] zhanghaou commented on a change in pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
zhanghaou commented on a change in pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#discussion_r445615755



##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java
##########
@@ -453,19 +453,23 @@
     /**
      * Update managed-ledger's properties.
      *
-     * @param properties key-values of properties
+     * @param properties  key-values of properties
+     * @param isOverwrite set true to overwrite original value of managed-ledger's properties; set false to append the
+     *                    properties to managed-ledger's properties.
      */
-    void setProperties(Map<String, String> properties) throws InterruptedException;
+    void setProperties(Map<String, String> properties, boolean isOverwrite) throws InterruptedException;

Review comment:
       OK. Thanks for the suggestion, I will add these APIs.

##########
File path: managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java
##########
@@ -1174,43 +1174,49 @@ public void testSetProperties() throws Exception {
         properties.put("key1", "value1");
         properties.put("key2", "value2");
         properties.put("key3", "value3");
-        ledger.setProperties(properties);
+        ledger.setProperties(properties, true);
         assertEquals(ledger.getProperties(), properties);
 
         Map<String, String> newProperties = new HashMap<>();
         newProperties.put("key4", "value4");
         newProperties.put("key5", "value5");
         newProperties.put("key6", "value6");
-        ledger.setProperties(newProperties);
+        ledger.setProperties(newProperties, true);
+        assertEquals(ledger.getProperties(), newProperties);
+
+        ledger.setProperties(properties, false);
+        newProperties.putAll(properties);

Review comment:
       OK.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] zhanghaou commented on pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
zhanghaou commented on pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#issuecomment-650186159


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] zhanghaou commented on a change in pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
zhanghaou commented on a change in pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#discussion_r446480928



##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java
##########
@@ -450,6 +450,23 @@
      */
     Map<String, String> getProperties();
 
+    /**
+     * Add key-value to propertiesMap.
+     *
+     * @param key
+     * @param value
+     * @throws InterruptedException
+     */
+    void setProperty(String key, String value) throws InterruptedException;

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] codelipenghui commented on pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#issuecomment-650779458


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] codelipenghui commented on pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#issuecomment-650221636


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] merlimat commented on a change in pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
merlimat commented on a change in pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#discussion_r446247774



##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
##########
@@ -3289,39 +3315,34 @@ public void setPropertiesFailed(ManagedLedgerException exception, Object ctx) {
         latch.await();
     }
 
-    @Override
-    public void asyncSetProperties(Map<String, String> properties, final SetPropertiesCallback callback, Object ctx) {
-        store.getManagedLedgerInfo(name, false, new MetaStoreCallback<ManagedLedgerInfo>() {
+    private void asyncUpdateProperties(Map<String, String> properties, boolean isOverwrite, boolean isDelete,
+        String deleteKey, final SetPropertiesCallback callback, Object ctx) {
+        if (!metadataMutex.tryLock()) {
+            // Defer update for later
+            scheduledExecutor.schedule(() -> asyncUpdateProperties(properties, isOverwrite, isDelete, deleteKey,
+                callback, ctx), 100, TimeUnit.MILLISECONDS);
+            return;
+        }
+        if (isDelete) {
+            propertiesMap.remove(deleteKey);
+        } else if (isOverwrite) {

Review comment:
       I still think the `setProperties()` should not blindly overwrite the current map, rather just overwrite the individual keys. It's just a convenience to have 1 metadata op to set multiple keys.
   
   Otherwise it would be a `replaceProperties()`, but I don't see how would that be more useful than a `setProperties()`.

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
##########
@@ -3270,10 +3274,32 @@ public long getOffloadedSize() {
         return propertiesMap;
     }
 
+    @Override
+    public void setProperty(String key, String value) throws InterruptedException {
+        Map<String, String> map = new HashMap<>();
+        map.put(key, value);
+        updateProperties(map, false, false, null);
+    }
+
+    @Override
+    public void deleteProperty(String key) throws InterruptedException {
+        updateProperties(null, false, true, key);
+    }
+
     @Override
     public void setProperties(Map<String, String> properties) throws InterruptedException {
+        updateProperties(properties, true, false, null);
+    }
+
+    @Override
+    public void asyncSetProperties(Map<String, String> properties, final SetPropertiesCallback callback, Object ctx) {
+        asyncUpdateProperties(properties, true, false, null, callback, ctx);
+    }
+
+    private void updateProperties(Map<String, String> properties, boolean isOverwrite, boolean isDelete,

Review comment:
       How is this reporting a failure in the operation?

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java
##########
@@ -450,6 +450,23 @@
      */
     Map<String, String> getProperties();
 
+    /**
+     * Add key-value to propertiesMap.
+     *
+     * @param key
+     * @param value
+     * @throws InterruptedException
+     */
+    void setProperty(String key, String value) throws InterruptedException;

Review comment:
       We also need the `asyncSetProperty()` and `asyncDeleteProperty()` versions.
   
   The throws, should also include `ManagedLedgedException`

##########
File path: managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java
##########
@@ -1177,40 +1177,59 @@ public void testSetProperties() throws Exception {
         ledger.setProperties(properties);
         assertEquals(ledger.getProperties(), properties);
 
+        properties.put("key4", "value4");
+        ledger.setProperty("key4", "value4");
+        assertEquals(ledger.getProperties(), properties);
+
+        ledger.deleteProperty("key4");
+        properties.remove("key4");
+        assertEquals(ledger.getProperties(), properties);
+
         Map<String, String> newProperties = new HashMap<>();
-        newProperties.put("key4", "value4");
         newProperties.put("key5", "value5");
         newProperties.put("key6", "value6");
         ledger.setProperties(newProperties);
         assertEquals(ledger.getProperties(), newProperties);
     }
 
     @Test
-    public void testAsyncSetProperties() throws Exception {
-        final CountDownLatch latch = new CountDownLatch(1);
-        ManagedLedger ledger = factory.open("my_test_ledger");
-        Map<String, String> properties = new HashMap<>();
-        properties.put("key1", "value1");
-        properties.put("key2", "value2");
-        properties.put("key3", "value3");
-        ledger.setProperties(properties);
-        Map<String, String> newProperties = new HashMap<>();
-        newProperties.put("key4", "value4");
-        newProperties.put("key5", "value5");
-        newProperties.put("key6", "value6");
-        ledger.asyncSetProperties(newProperties, new AsyncCallbacks.SetPropertiesCallback() {
-            @Override
-            public void setPropertiesComplete(Map<String, String> properties, Object ctx) {
-                latch.countDown();
+    public void testConcurrentAsyncSetProperties() throws Exception {
+        final CountDownLatch latch = new CountDownLatch(1000);
+        ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(1));
+        Executor executor = Executors.newCachedThreadPool();
+        executor.execute(()->{
+            try {
+                for (int i = 0; i < 100; i++) {
+                    ledger.addEntry("data".getBytes(Encoding));
+                    Thread.sleep(300);
+                }
+            } catch (Exception e) {
+                fail(e.getMessage());

Review comment:
       Since this is not in the main thread, it will not fail the test itself.
   
   We don't need to do it from different threads. It would be enough to do something like: 
   
   ```java
   ledger.addEntry(e1);
   ledger.setProperty("x", "xx");
   ledger.addEntry(e2);
   // ...
   ```

##########
File path: managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java
##########
@@ -1177,40 +1177,59 @@ public void testSetProperties() throws Exception {
         ledger.setProperties(properties);
         assertEquals(ledger.getProperties(), properties);
 
+        properties.put("key4", "value4");
+        ledger.setProperty("key4", "value4");
+        assertEquals(ledger.getProperties(), properties);
+
+        ledger.deleteProperty("key4");
+        properties.remove("key4");
+        assertEquals(ledger.getProperties(), properties);
+
         Map<String, String> newProperties = new HashMap<>();
-        newProperties.put("key4", "value4");
         newProperties.put("key5", "value5");
         newProperties.put("key6", "value6");
         ledger.setProperties(newProperties);
         assertEquals(ledger.getProperties(), newProperties);
     }
 
     @Test
-    public void testAsyncSetProperties() throws Exception {
-        final CountDownLatch latch = new CountDownLatch(1);
-        ManagedLedger ledger = factory.open("my_test_ledger");
-        Map<String, String> properties = new HashMap<>();
-        properties.put("key1", "value1");
-        properties.put("key2", "value2");
-        properties.put("key3", "value3");
-        ledger.setProperties(properties);
-        Map<String, String> newProperties = new HashMap<>();
-        newProperties.put("key4", "value4");
-        newProperties.put("key5", "value5");
-        newProperties.put("key6", "value6");
-        ledger.asyncSetProperties(newProperties, new AsyncCallbacks.SetPropertiesCallback() {
-            @Override
-            public void setPropertiesComplete(Map<String, String> properties, Object ctx) {
-                latch.countDown();
+    public void testConcurrentAsyncSetProperties() throws Exception {
+        final CountDownLatch latch = new CountDownLatch(1000);
+        ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(1));
+        Executor executor = Executors.newCachedThreadPool();

Review comment:
       shutdown executor at the end




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] zhanghaou commented on pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
zhanghaou commented on pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#issuecomment-650744765


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] merlimat commented on a change in pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
merlimat commented on a change in pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#discussion_r445590129



##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java
##########
@@ -453,19 +453,23 @@
     /**
      * Update managed-ledger's properties.
      *
-     * @param properties key-values of properties
+     * @param properties  key-values of properties
+     * @param isOverwrite set true to overwrite original value of managed-ledger's properties; set false to append the
+     *                    properties to managed-ledger's properties.
      */
-    void setProperties(Map<String, String> properties) throws InterruptedException;
+    void setProperties(Map<String, String> properties, boolean isOverwrite) throws InterruptedException;

Review comment:
       Using `isOverwrite` feels a bit weird compared to similar properties APIs.
   
   What about having: 
    * `setProperty()` 
    * `setProperties()` (with `isOverwrite==false` behavior)
    * `deleteProperty()` 

##########
File path: managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java
##########
@@ -1174,43 +1174,49 @@ public void testSetProperties() throws Exception {
         properties.put("key1", "value1");
         properties.put("key2", "value2");
         properties.put("key3", "value3");
-        ledger.setProperties(properties);
+        ledger.setProperties(properties, true);
         assertEquals(ledger.getProperties(), properties);
 
         Map<String, String> newProperties = new HashMap<>();
         newProperties.put("key4", "value4");
         newProperties.put("key5", "value5");
         newProperties.put("key6", "value6");
-        ledger.setProperties(newProperties);
+        ledger.setProperties(newProperties, true);
+        assertEquals(ledger.getProperties(), newProperties);
+
+        ledger.setProperties(properties, false);
+        newProperties.putAll(properties);

Review comment:
       Please also try to add entries in the ledger, to verify that we're not fenced off from the metadata updates. 
   
   Change the settings to force multiple rollovers, eg: 
   
   ```java
   ManagedLedger ledger = factory.open("my_test_ledger",
        new ManagedLedgerConfig().setMaxEntriesPerLedger(1));
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] merlimat merged pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
merlimat merged pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] zhanghaou commented on a change in pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
zhanghaou commented on a change in pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#discussion_r446481027



##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
##########
@@ -3270,10 +3274,32 @@ public long getOffloadedSize() {
         return propertiesMap;
     }
 
+    @Override
+    public void setProperty(String key, String value) throws InterruptedException {
+        Map<String, String> map = new HashMap<>();
+        map.put(key, value);
+        updateProperties(map, false, false, null);
+    }
+
+    @Override
+    public void deleteProperty(String key) throws InterruptedException {
+        updateProperties(null, false, true, key);
+    }
+
     @Override
     public void setProperties(Map<String, String> properties) throws InterruptedException {
+        updateProperties(properties, true, false, null);
+    }
+
+    @Override
+    public void asyncSetProperties(Map<String, String> properties, final SetPropertiesCallback callback, Object ctx) {
+        asyncUpdateProperties(properties, true, false, null, callback, ctx);
+    }
+
+    private void updateProperties(Map<String, String> properties, boolean isOverwrite, boolean isDelete,

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] zhanghaou commented on pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
zhanghaou commented on pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#issuecomment-650674337


   Please take a look again. @merlimat @jiazhai @codelipenghui @sijie 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] zhanghaou commented on a change in pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
zhanghaou commented on a change in pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#discussion_r446480959



##########
File path: managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java
##########
@@ -1177,40 +1177,59 @@ public void testSetProperties() throws Exception {
         ledger.setProperties(properties);
         assertEquals(ledger.getProperties(), properties);
 
+        properties.put("key4", "value4");
+        ledger.setProperty("key4", "value4");
+        assertEquals(ledger.getProperties(), properties);
+
+        ledger.deleteProperty("key4");
+        properties.remove("key4");
+        assertEquals(ledger.getProperties(), properties);
+
         Map<String, String> newProperties = new HashMap<>();
-        newProperties.put("key4", "value4");
         newProperties.put("key5", "value5");
         newProperties.put("key6", "value6");
         ledger.setProperties(newProperties);
         assertEquals(ledger.getProperties(), newProperties);
     }
 
     @Test
-    public void testAsyncSetProperties() throws Exception {
-        final CountDownLatch latch = new CountDownLatch(1);
-        ManagedLedger ledger = factory.open("my_test_ledger");
-        Map<String, String> properties = new HashMap<>();
-        properties.put("key1", "value1");
-        properties.put("key2", "value2");
-        properties.put("key3", "value3");
-        ledger.setProperties(properties);
-        Map<String, String> newProperties = new HashMap<>();
-        newProperties.put("key4", "value4");
-        newProperties.put("key5", "value5");
-        newProperties.put("key6", "value6");
-        ledger.asyncSetProperties(newProperties, new AsyncCallbacks.SetPropertiesCallback() {
-            @Override
-            public void setPropertiesComplete(Map<String, String> properties, Object ctx) {
-                latch.countDown();
+    public void testConcurrentAsyncSetProperties() throws Exception {
+        final CountDownLatch latch = new CountDownLatch(1000);
+        ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(1));
+        Executor executor = Executors.newCachedThreadPool();
+        executor.execute(()->{
+            try {
+                for (int i = 0; i < 100; i++) {
+                    ledger.addEntry("data".getBytes(Encoding));
+                    Thread.sleep(300);
+                }
+            } catch (Exception e) {
+                fail(e.getMessage());

Review comment:
       Done.

##########
File path: managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java
##########
@@ -1177,40 +1177,59 @@ public void testSetProperties() throws Exception {
         ledger.setProperties(properties);
         assertEquals(ledger.getProperties(), properties);
 
+        properties.put("key4", "value4");
+        ledger.setProperty("key4", "value4");
+        assertEquals(ledger.getProperties(), properties);
+
+        ledger.deleteProperty("key4");
+        properties.remove("key4");
+        assertEquals(ledger.getProperties(), properties);
+
         Map<String, String> newProperties = new HashMap<>();
-        newProperties.put("key4", "value4");
         newProperties.put("key5", "value5");
         newProperties.put("key6", "value6");
         ledger.setProperties(newProperties);
         assertEquals(ledger.getProperties(), newProperties);
     }
 
     @Test
-    public void testAsyncSetProperties() throws Exception {
-        final CountDownLatch latch = new CountDownLatch(1);
-        ManagedLedger ledger = factory.open("my_test_ledger");
-        Map<String, String> properties = new HashMap<>();
-        properties.put("key1", "value1");
-        properties.put("key2", "value2");
-        properties.put("key3", "value3");
-        ledger.setProperties(properties);
-        Map<String, String> newProperties = new HashMap<>();
-        newProperties.put("key4", "value4");
-        newProperties.put("key5", "value5");
-        newProperties.put("key6", "value6");
-        ledger.asyncSetProperties(newProperties, new AsyncCallbacks.SetPropertiesCallback() {
-            @Override
-            public void setPropertiesComplete(Map<String, String> properties, Object ctx) {
-                latch.countDown();
+    public void testConcurrentAsyncSetProperties() throws Exception {
+        final CountDownLatch latch = new CountDownLatch(1000);
+        ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(1));
+        Executor executor = Executors.newCachedThreadPool();

Review comment:
       Done.

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
##########
@@ -3289,39 +3315,34 @@ public void setPropertiesFailed(ManagedLedgerException exception, Object ctx) {
         latch.await();
     }
 
-    @Override
-    public void asyncSetProperties(Map<String, String> properties, final SetPropertiesCallback callback, Object ctx) {
-        store.getManagedLedgerInfo(name, false, new MetaStoreCallback<ManagedLedgerInfo>() {
+    private void asyncUpdateProperties(Map<String, String> properties, boolean isOverwrite, boolean isDelete,
+        String deleteKey, final SetPropertiesCallback callback, Object ctx) {
+        if (!metadataMutex.tryLock()) {
+            // Defer update for later
+            scheduledExecutor.schedule(() -> asyncUpdateProperties(properties, isOverwrite, isDelete, deleteKey,
+                callback, ctx), 100, TimeUnit.MILLISECONDS);
+            return;
+        }
+        if (isDelete) {
+            propertiesMap.remove(deleteKey);
+        } else if (isOverwrite) {

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] zhanghaou commented on pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
zhanghaou commented on pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#issuecomment-649535507


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] zhanghaou commented on pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
zhanghaou commented on pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#issuecomment-650742818


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [pulsar] jiazhai commented on pull request #7357: Fix bug related to managedLedger properties

Posted by GitBox <gi...@apache.org>.
jiazhai commented on pull request #7357:
URL: https://github.com/apache/pulsar/pull/7357#issuecomment-650649769


   @zhanghaou to help resolve the latest comments


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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