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/12/23 20:45:42 UTC

[GitHub] [pulsar] sijie commented on a change in pull request #9039: [Feature] Introduce continuous offset for pulsar

sijie commented on a change in pull request #9039:
URL: https://github.com/apache/pulsar/pull/9039#discussion_r548143593



##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java
##########
@@ -520,4 +587,14 @@ void asyncSetProperties(Map<String, String> properties, final AsyncCallbacks.Upd
      * Roll current ledger if it is full
      */
     void rollCurrentLedgerIfFull();
+
+    /**
+     * Find position by sequenceId.
+     * */
+    CompletableFuture<PositionImpl> asyncFindPosition(com.google.common.base.Predicate<Entry> predicate);

Review comment:
       ```suggestion
       CompletableFuture<Position> asyncFindPosition(com.google.common.base.Predicate<Entry> predicate);
   ```

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
##########
@@ -550,6 +566,11 @@ public Position addEntry(byte[] data) throws InterruptedException, ManagedLedger
         return addEntry(data, 0, data.length);
     }
 
+    @Override
+    public Position addEntry(byte[] data, int batchSize) throws InterruptedException, ManagedLedgerException {
+        return addEntry(data, batchSize,0, data.length);

Review comment:
       ```suggestion
           return addEntry(data, batchSize, 0, data.length);
   ```

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java
##########
@@ -520,4 +587,14 @@ void asyncSetProperties(Map<String, String> properties, final AsyncCallbacks.Upd
      * Roll current ledger if it is full
      */
     void rollCurrentLedgerIfFull();
+
+    /**
+     * Find position by sequenceId.
+     * */
+    CompletableFuture<PositionImpl> asyncFindPosition(com.google.common.base.Predicate<Entry> predicate);

Review comment:
       ```suggestion
       CompletableFuture<PositionImpl> asyncFindPosition(com.google.common.base.Predicate<Entry> predicate);
   ```
   
   Can we use the java.util.Predicate?

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/interceptor/ManagedLedgerInterceptor.java
##########
@@ -0,0 +1,34 @@
+/**
+ * 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.mledger.interceptor;
+
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.mledger.impl.OpAddEntry;
+
+import java.util.Map;
+
+/**
+ * Interceptor for ManagedLedger.
+ * */
+public interface ManagedLedgerInterceptor {

Review comment:
       Can you add the interface annotations to the new interface? See https://github.com/apache/pulsar/wiki/PIP-72%3A-Introduce-Pulsar-Interface-Taxonomy%3A-Audience-and-Stability-Classification

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/interceptor/ManagedLedgerInterceptor.java
##########
@@ -0,0 +1,34 @@
+/**
+ * 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.mledger.interceptor;
+
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.mledger.impl.OpAddEntry;
+
+import java.util.Map;
+
+/**
+ * Interceptor for ManagedLedger.
+ * */
+public interface ManagedLedgerInterceptor {
+    OpAddEntry beforeAddEntry(OpAddEntry op, int batchSize);

Review comment:
       Can you add javadoc to the methods?

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
##########
@@ -672,8 +747,27 @@ private synchronized void internalAsyncAddEntry(OpAddEntry addOperation) {
                 addOperation.setCloseWhenDone(true);
                 STATE_UPDATER.set(this, State.ClosingLedger);
             }
+            // interceptor entry before add to bookie
+            if (beforeAddEntry(addOperation)) {
+                addOperation.initiate();
+            }
+        }
+    }
 
-            addOperation.initiate();
+    private boolean beforeAddEntry(OpAddEntry addOperation) {
+        // if no interceptor, just return true to make sure addOperation will be initiate()
+        if (managedLedgerInterceptor == null) {
+            return true;
+        }
+        try {
+            managedLedgerInterceptor.beforeAddEntry(addOperation, addOperation.getBatchSize());
+            return true;
+        } catch (Exception e) {
+            addOperation.failed(
+                    new ManagedLedgerInterceptException("Interceptor managed ledger before add to bookie failed."));
+            ReferenceCountUtil.release(addOperation.data);
+            log.error("[{}] Failed to interceptor entry before add to bookie.", name, e);

Review comment:
       ```suggestion
               log.error("[{}] Failed to intercept adding an entry to bookie.", name, e);
   ```

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
##########
@@ -357,10 +357,7 @@ public void publishMessage(ByteBuf headersAndPayload, PublishContext publishCont
                 messageDeduplication.isDuplicate(publishContext, headersAndPayload);
         switch (status) {
             case NotDup:
-                // intercept headersAndPayload and add entry metadata
-                if (appendBrokerEntryMetadata(headersAndPayload, publishContext)) {
-                    ledger.asyncAddEntry(headersAndPayload, this, publishContext);
-                }
+                ledger.asyncAddEntry(headersAndPayload, this, publishContext);

Review comment:
       How is the `batchSize` passed to `asyncAddEntry`? I failed to see how did you do that in this pull request.
   
   At the same time, I think `batchSize` is not a good term. If I understand this correctly, it should be `numberOfMessages`, correct?

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImpl.java
##########
@@ -0,0 +1,113 @@
+/**
+ * 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.pulsar.broker.intercept;
+
+import java.util.Map;
+import java.util.Set;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.api.LedgerEntries;
+import org.apache.bookkeeper.client.api.LedgerEntry;
+import org.apache.bookkeeper.mledger.impl.OpAddEntry;
+import org.apache.bookkeeper.mledger.interceptor.ManagedLedgerInterceptor;
+import org.apache.pulsar.common.api.proto.PulsarApi;
+import org.apache.pulsar.common.intercept.AppendOffsetMetadataInterceptor;
+import org.apache.pulsar.common.intercept.BrokerEntryMetadataInterceptor;
+import org.apache.pulsar.common.protocol.Commands;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ManagedLedgerInterceptorImpl implements ManagedLedgerInterceptor {
+    private static final Logger log = LoggerFactory.getLogger(ManagedLedgerInterceptorImpl.class);
+    private static final String OFFSET = "offset";

Review comment:
       kafka `offset` is not a very good term to be used for describing the index of a message within a log stream. Instead, I would suggest calling it `index` or `logIndex` similar to the term used in the Raft algorithm (https://cs.stackexchange.com/questions/97542/raft-algorithm-whats-the-meaning-of-concept-index).




----------------------------------------------------------------
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