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 2021/11/30 09:13:53 UTC

[GitHub] [pulsar] Jason918 commented on a change in pull request #13043: Transparent batching of ZK operations

Jason918 commented on a change in pull request #13043:
URL: https://github.com/apache/pulsar/pull/13043#discussion_r759056059



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java
##########
@@ -0,0 +1,153 @@
+/**
+ * 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.metadata.impl.batching;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.metadata.api.GetResult;
+import org.apache.pulsar.metadata.api.MetadataStoreConfig;
+import org.apache.pulsar.metadata.api.Stat;
+import org.apache.pulsar.metadata.api.extended.CreateOption;
+import org.apache.pulsar.metadata.impl.AbstractMetadataStore;
+import org.jctools.queues.MessagePassingQueue;
+import org.jctools.queues.MpscUnboundedArrayQueue;
+
+@Slf4j
+public abstract class AbstractBatchedMetadataStore extends AbstractMetadataStore {
+
+    private final ScheduledFuture<?> scheduledTask;
+    private final MessagePassingQueue<MetadataOp> readOps;
+    private final MessagePassingQueue<MetadataOp> writeOps;
+
+    private final AtomicBoolean flushInProgress = new AtomicBoolean(false);
+
+    private final boolean enabled;
+    private final int maxDelayMillis;
+    private final int maxOperations;
+    private final int maxSize;
+
+    protected AbstractBatchedMetadataStore(MetadataStoreConfig conf) {
+        super();
+
+        this.enabled = conf.isBatchingEnabled();
+        this.maxDelayMillis = conf.getBatchingMaxDelayMillis();
+        this.maxOperations = conf.getBatchingMaxOperations();
+        this.maxSize = conf.getBatchingMaxSizeKb() * 1_024;
+
+        readOps = new MpscUnboundedArrayQueue<>(10_000);
+        writeOps = new MpscUnboundedArrayQueue<>(10_000);
+        if (enabled) {
+            scheduledTask = executor.scheduleAtFixedRate(this::flush, 5, 5, TimeUnit.MILLISECONDS);
+        } else {
+            scheduledTask = null;
+        }
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (scheduledTask != null) {

Review comment:
       Should we fails all ops in readOps and writeOps here?

##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java
##########
@@ -0,0 +1,153 @@
+/**
+ * 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.metadata.impl.batching;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.metadata.api.GetResult;
+import org.apache.pulsar.metadata.api.MetadataStoreConfig;
+import org.apache.pulsar.metadata.api.Stat;
+import org.apache.pulsar.metadata.api.extended.CreateOption;
+import org.apache.pulsar.metadata.impl.AbstractMetadataStore;
+import org.jctools.queues.MessagePassingQueue;
+import org.jctools.queues.MpscUnboundedArrayQueue;
+
+@Slf4j
+public abstract class AbstractBatchedMetadataStore extends AbstractMetadataStore {
+
+    private final ScheduledFuture<?> scheduledTask;
+    private final MessagePassingQueue<MetadataOp> readOps;
+    private final MessagePassingQueue<MetadataOp> writeOps;
+
+    private final AtomicBoolean flushInProgress = new AtomicBoolean(false);
+
+    private final boolean enabled;
+    private final int maxDelayMillis;
+    private final int maxOperations;
+    private final int maxSize;
+
+    protected AbstractBatchedMetadataStore(MetadataStoreConfig conf) {
+        super();
+
+        this.enabled = conf.isBatchingEnabled();
+        this.maxDelayMillis = conf.getBatchingMaxDelayMillis();
+        this.maxOperations = conf.getBatchingMaxOperations();
+        this.maxSize = conf.getBatchingMaxSizeKb() * 1_024;
+
+        readOps = new MpscUnboundedArrayQueue<>(10_000);

Review comment:
       No need init readOps and writeOps if enabled is false.
   `MpscUnboundedArrayQueue` creates a buffer inside.

##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java
##########
@@ -0,0 +1,153 @@
+/**
+ * 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.metadata.impl.batching;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.metadata.api.GetResult;
+import org.apache.pulsar.metadata.api.MetadataStoreConfig;
+import org.apache.pulsar.metadata.api.Stat;
+import org.apache.pulsar.metadata.api.extended.CreateOption;
+import org.apache.pulsar.metadata.impl.AbstractMetadataStore;
+import org.jctools.queues.MessagePassingQueue;
+import org.jctools.queues.MpscUnboundedArrayQueue;
+
+@Slf4j
+public abstract class AbstractBatchedMetadataStore extends AbstractMetadataStore {
+
+    private final ScheduledFuture<?> scheduledTask;
+    private final MessagePassingQueue<MetadataOp> readOps;
+    private final MessagePassingQueue<MetadataOp> writeOps;
+
+    private final AtomicBoolean flushInProgress = new AtomicBoolean(false);
+
+    private final boolean enabled;
+    private final int maxDelayMillis;
+    private final int maxOperations;
+    private final int maxSize;
+
+    protected AbstractBatchedMetadataStore(MetadataStoreConfig conf) {
+        super();
+
+        this.enabled = conf.isBatchingEnabled();
+        this.maxDelayMillis = conf.getBatchingMaxDelayMillis();
+        this.maxOperations = conf.getBatchingMaxOperations();
+        this.maxSize = conf.getBatchingMaxSizeKb() * 1_024;
+
+        readOps = new MpscUnboundedArrayQueue<>(10_000);
+        writeOps = new MpscUnboundedArrayQueue<>(10_000);
+        if (enabled) {
+            scheduledTask = executor.scheduleAtFixedRate(this::flush, 5, 5, TimeUnit.MILLISECONDS);

Review comment:
       should be `maxDelayMillis`?

##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/OpPut.java
##########
@@ -0,0 +1,52 @@
+/**
+ * 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.metadata.impl.batching;
+
+import java.util.EnumSet;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.pulsar.metadata.api.Stat;
+import org.apache.pulsar.metadata.api.extended.CreateOption;
+
+@Data
+@AllArgsConstructor
+public class OpPut implements MetadataOp {
+    private final String path;
+    private final byte[] data;
+    private final Optional<Long> optExpectedVersion;
+    private final EnumSet<CreateOption> options;
+
+    private final CompletableFuture<Stat> future = new CompletableFuture<>();
+
+    public boolean isEphemeral() {
+        return options.contains(CreateOption.Ephemeral);
+    }
+
+    @Override
+    public Type getType() {
+        return Type.PUT;
+    }
+
+    @Override
+    public int size() {
+        return path.length() + data.length;

Review comment:
       This `data` could be null?  

##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java
##########
@@ -0,0 +1,153 @@
+/**
+ * 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.metadata.impl.batching;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.metadata.api.GetResult;
+import org.apache.pulsar.metadata.api.MetadataStoreConfig;
+import org.apache.pulsar.metadata.api.Stat;
+import org.apache.pulsar.metadata.api.extended.CreateOption;
+import org.apache.pulsar.metadata.impl.AbstractMetadataStore;
+import org.jctools.queues.MessagePassingQueue;
+import org.jctools.queues.MpscUnboundedArrayQueue;
+
+@Slf4j
+public abstract class AbstractBatchedMetadataStore extends AbstractMetadataStore {
+
+    private final ScheduledFuture<?> scheduledTask;
+    private final MessagePassingQueue<MetadataOp> readOps;
+    private final MessagePassingQueue<MetadataOp> writeOps;
+
+    private final AtomicBoolean flushInProgress = new AtomicBoolean(false);
+
+    private final boolean enabled;
+    private final int maxDelayMillis;
+    private final int maxOperations;
+    private final int maxSize;
+
+    protected AbstractBatchedMetadataStore(MetadataStoreConfig conf) {
+        super();
+
+        this.enabled = conf.isBatchingEnabled();
+        this.maxDelayMillis = conf.getBatchingMaxDelayMillis();
+        this.maxOperations = conf.getBatchingMaxOperations();
+        this.maxSize = conf.getBatchingMaxSizeKb() * 1_024;
+
+        readOps = new MpscUnboundedArrayQueue<>(10_000);
+        writeOps = new MpscUnboundedArrayQueue<>(10_000);
+        if (enabled) {
+            scheduledTask = executor.scheduleAtFixedRate(this::flush, 5, 5, TimeUnit.MILLISECONDS);
+        } else {
+            scheduledTask = null;
+        }
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (scheduledTask != null) {
+            scheduledTask.cancel(true);
+        }
+        super.close();
+    }
+
+    private void flush() {
+        while (!readOps.isEmpty()) {
+            List<MetadataOp> ops = new ArrayList<>();
+            readOps.drain(ops::add, maxOperations);
+            batchOperation(ops);
+        }
+
+        while (!writeOps.isEmpty()) {
+            int batchSize = 0;
+
+            List<MetadataOp> ops = new ArrayList<>();
+            for (int i = 0; i < maxOperations; i++) {
+                MetadataOp op = writeOps.peek();
+                if (op == null) {
+                    break;
+                }
+
+                if (i > 0 && (batchSize + op.size()) > maxSize) {
+                    // We have already reached the max size, so flush the current batch
+                    break;
+                }
+
+                batchSize += op.size();
+                ops.add(writeOps.poll());
+            }
+            batchOperation(ops);
+        }
+
+        flushInProgress.set(false);
+    }
+
+    @Override
+    public final CompletableFuture<Optional<GetResult>> storeGet(String path) {
+        OpGet op = new OpGet(path);
+        enqueue(readOps, op);
+        return op.getFuture();
+    }
+
+    @Override
+    protected final CompletableFuture<List<String>> getChildrenFromStore(String path) {
+        OpGetChildren op = new OpGetChildren(path);
+        enqueue(readOps, op);
+        return op.getFuture();
+    }
+
+    @Override
+    protected final CompletableFuture<Void> storeDelete(String path, Optional<Long> expectedVersion) {
+        OpDelete op = new OpDelete(path, expectedVersion);
+        enqueue(writeOps, op);
+        return op.getFuture();
+    }
+
+    @Override
+    protected final CompletableFuture<Stat> storePut(String path, byte[] data, Optional<Long> optExpectedVersion,
+                                               EnumSet<CreateOption> options) {
+        OpPut op = new OpPut(path, data, optExpectedVersion, options);
+        enqueue(writeOps, op);
+        return op.getFuture();
+    }
+
+    private void enqueue(MessagePassingQueue queue, MetadataOp op) {
+        if (enabled) {
+            if (!queue.offer(op)) {
+                op.getFuture().completeExceptionally(new IllegalStateException("metadata queue is full"));

Review comment:
       Fallback to execute it directly?




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

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org