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



##########
File path: 
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java
##########
@@ -0,0 +1,423 @@
+/**
+ * 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;
+
+import io.etcd.jetcd.ByteSequence;
+import io.etcd.jetcd.Client;
+import io.etcd.jetcd.KV;
+import io.etcd.jetcd.KeyValue;
+import io.etcd.jetcd.Txn;
+import io.etcd.jetcd.kv.DeleteResponse;
+import io.etcd.jetcd.kv.GetResponse;
+import io.etcd.jetcd.kv.PutResponse;
+import io.etcd.jetcd.lease.LeaseKeepAliveResponse;
+import io.etcd.jetcd.op.Cmp;
+import io.etcd.jetcd.op.CmpTarget;
+import io.etcd.jetcd.op.Op;
+import io.etcd.jetcd.options.DeleteOption;
+import io.etcd.jetcd.options.GetOption;
+import io.etcd.jetcd.options.PutOption;
+import io.etcd.jetcd.options.WatchOption;
+import io.etcd.jetcd.support.CloseableClient;
+import io.etcd.jetcd.watch.WatchEvent;
+import io.etcd.jetcd.watch.WatchResponse;
+import io.grpc.Status;
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+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.MetadataStoreException;
+import org.apache.pulsar.metadata.api.Notification;
+import org.apache.pulsar.metadata.api.NotificationType;
+import org.apache.pulsar.metadata.api.Stat;
+import org.apache.pulsar.metadata.api.extended.CreateOption;
+import org.apache.pulsar.metadata.api.extended.SessionEvent;
+import org.apache.pulsar.metadata.impl.batching.AbstractBatchedMetadataStore;
+import org.apache.pulsar.metadata.impl.batching.MetadataOp;
+import org.apache.pulsar.metadata.impl.batching.OpDelete;
+import org.apache.pulsar.metadata.impl.batching.OpGet;
+import org.apache.pulsar.metadata.impl.batching.OpGetChildren;
+import org.apache.pulsar.metadata.impl.batching.OpPut;
+import org.apache.zookeeper.AddWatchMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+
+@Slf4j
+public class EtcdMetadataStore extends AbstractBatchedMetadataStore {
+
+    static final String ETCD_SCHEME_IDENTIFIER = "etcd:";
+
+    private final int leaseTTLSeconds;
+    private final Client client;
+    private final KV kv;
+    private volatile long leaseId;
+    private volatile CloseableClient leaseClient;
+    private final EtcdSessionWatcher sessionWatcher;
+
+    public EtcdMetadataStore(String metadataURL, MetadataStoreConfig conf, 
boolean enableSessionWatcher)
+            throws MetadataStoreException {
+        super(conf);
+
+        this.leaseTTLSeconds = conf.getSessionTimeoutMillis() / 1000;
+        String etcdUrl = metadataURL.replaceFirst(ETCD_SCHEME_IDENTIFIER, "");
+
+        try {
+            this.client = Client.builder().endpoints(etcdUrl).build();
+            this.kv = client.getKVClient();
+            this.client.getWatchClient().watch(ByteSequence.from("\0", 
StandardCharsets.UTF_8),
+                    WatchOption.newBuilder()
+                            .withPrefix(ByteSequence.from("/", 
StandardCharsets.UTF_8))
+                            .build(), this::handleWatchResponse);
+            this.sessionWatcher =
+                    new EtcdSessionWatcher(client, 
conf.getSessionTimeoutMillis(), this::receivedSessionEvent);
+
+            // Ensure the lease is created when we start
+            this.createLease(false).join();
+        } catch (Exception e) {
+            throw new MetadataStoreException(e);
+        }
+    }
+
+    @Override
+    public void close() throws Exception {
+        super.close();
+
+        if (sessionWatcher != null) {
+            sessionWatcher.close();
+        }
+
+        leaseClient.close();
+        client.getLeaseClient().revoke(leaseId);
+
+        kv.close();
+        client.close();
+    }
+
+    private static final GetOption EXISTS_GET_OPTION = 
GetOption.newBuilder().withCountOnly(true).build();
+    private static final GetOption SINGLE_GET_OPTION = 
GetOption.newBuilder().withLimit(1).build();
+
+    @Override
+    protected CompletableFuture<Boolean> existsFromStore(String path) {
+        return kv.get(ByteSequence.from(path, StandardCharsets.UTF_8), 
EXISTS_GET_OPTION)
+                .thenApply(gr -> gr.getCount() == 1);
+    }
+
+    @Override
+    protected CompletableFuture<Stat> storePut(String path, byte[] data, 
Optional<Long> optExpectedVersion,
+                                               EnumSet<CreateOption> options) {
+        if (!options.contains(CreateOption.Sequential)) {
+            return super.storePut(path, data, optExpectedVersion, options);
+        } else {
+            // First get the version from parent
+            String parent = parent(path);
+            return super.storePut(parent, new byte[0], Optional.empty(), 
EnumSet.noneOf(CreateOption.class))
+                    // Then create the unique key with the version added in 
the path
+                    .thenCompose(stat -> super.storePut(path + 
stat.getVersion(), data, optExpectedVersion, options));
+        }
+    }
+
+    @Override
+    protected void batchOperation(List<MetadataOp> ops) {
+        try {
+            Txn txn = kv.txn();
+
+            // First, set all the conditions
+            ops.forEach(op -> {
+                switch (op.getType()) {
+                    case PUT: {
+                        OpPut put = op.asPut();
+                        ByteSequence key = ByteSequence.from(put.getPath(), 
StandardCharsets.UTF_8);
+                        if (put.getOptExpectedVersion().isPresent()) {
+                            long expectedVersion = 
put.getOptExpectedVersion().get();
+                            if (expectedVersion == -1L) {
+                                // Check that key does not exist
+                                txn.If(new Cmp(key, Cmp.Op.EQUAL, 
CmpTarget.createRevision(0)));
+                            } else {
+                                txn.If(new Cmp(key, Cmp.Op.EQUAL, 
CmpTarget.version(expectedVersion + 1)));
+                            }
+                        }
+                        break;
+                    }
+                    case DELETE: {
+                        OpDelete del = op.asDelete();
+                        ByteSequence key = ByteSequence.from(del.getPath(), 
StandardCharsets.UTF_8);
+                        if (del.getOptExpectedVersion().isPresent()) {
+                            txn.If(new Cmp(key, Cmp.Op.EQUAL,
+                                    
CmpTarget.version(del.getOptExpectedVersion().get() + 1)));
+                        }
+                        break;
+                    }
+                }
+            });
+
+            // Then the requests
+            ops.forEach(op -> {
+                switch (op.getType()) {
+                    case GET: {
+                        txn.Then(
+                                Op.get(ByteSequence.from(op.asGet().getPath(), 
StandardCharsets.UTF_8),
+                                        SINGLE_GET_OPTION));
+                        break;
+                    }
+                    case PUT: {
+                        OpPut put = op.asPut();
+                        ByteSequence key = ByteSequence.from(put.getPath(), 
StandardCharsets.UTF_8);
+                        if (!put.getFuture().isDone()) {
+                            PutOption.Builder b = PutOption.newBuilder()
+                                    .withPrevKV();
+
+                            if (put.isEphemeral()) {
+                                b.withLeaseId(leaseId);
+                            }
+
+                            txn.Then(Op.put(key, 
ByteSequence.from(put.getData()), b.build()));
+                        }
+                        break;
+                    }
+                    case DELETE: {
+                        OpDelete del = op.asDelete();
+                        ByteSequence key = ByteSequence.from(del.getPath(), 
StandardCharsets.UTF_8);
+                        txn.Then(Op.delete(key, DeleteOption.DEFAULT));
+                        break;
+                    }
+                    case GET_CHILDREN: {
+                        OpGetChildren opGetChildren = op.asGetChildren();
+                        String path = opGetChildren.getPath();
+
+                        ByteSequence firstKey =
+                                ByteSequence.from(path.equals("/") ? path : 
path + "/",
+                                        StandardCharsets.UTF_8);
+                        ByteSequence lastKey = 
ByteSequence.from(path.equals("/") ? "0" : path + "0",
+                                StandardCharsets.UTF_8); // '0' is 
lexicographically just after '/'
+
+                        txn.Then(Op.get(firstKey, GetOption.newBuilder()
+                                .withKeysOnly(true)
+                                .withSortField(GetOption.SortTarget.KEY)
+                                .withSortOrder(GetOption.SortOrder.ASCEND)
+                                .withRange(lastKey)
+                                .build()));
+                        break;
+                    }
+                }
+            });
+
+            txn.commit().thenAccept(txnResponse -> {
+                if (!txnResponse.isSucceeded()) {
+                    if (ops.size() > 1) {
+                        // Retry individually
+                        ops.forEach(o -> 
batchOperation(Collections.singletonList(o)));
+                    } else {
+                        ops.get(0).getFuture()
+                                .completeExceptionally(new 
MetadataStoreException.BadVersionException("Bad version"));
+                    }
+                    return;
+                }
+
+                int getIdx = 0;
+                int deletedIdx = 0;
+                int putIdx = 0;
+                for (MetadataOp op : ops) {
+                    switch (op.getType()) {
+                        case GET: {
+                            OpGet get = op.asGet();
+                            GetResponse gr = 
txnResponse.getGetResponses().get(getIdx++);
+                            if (gr.getCount() == 0) {
+                                get.getFuture().complete(Optional.empty());
+                            } else {
+                                KeyValue kv = gr.getKvs().get(0);
+                                boolean isEphemeral = kv.getLease() != 0;
+                                boolean createdBySelf = kv.getLease() == 
leaseId;
+                                get.getFuture().complete(Optional.of(
+                                                new GetResult(
+                                                        
kv.getValue().getBytes(),
+                                                        new 
Stat(get.getPath(), kv.getVersion() - 1, 0, 0, isEphemeral,
+                                                                createdBySelf)
+                                                )
+                                        )
+                                );
+                            }
+                            break;
+                        }
+                        case PUT: {
+                            OpPut put = op.asPut();
+                            PutResponse pr = 
txnResponse.getPutResponses().get(putIdx++);
+                            KeyValue prevKv = pr.getPrevKv();
+                            if (prevKv == null) {
+                                put.getFuture().complete(new 
Stat(put.getPath(),
+                                        0, 0, 0, put.isEphemeral(), true));
+                            } else {
+                                put.getFuture().complete(new 
Stat(put.getPath(),
+                                        prevKv.getVersion(), 0, 0, 
put.isEphemeral(), true));
+                            }
+
+                            break;
+                        }
+                        case DELETE: {
+                            OpDelete del = op.asDelete();
+                            DeleteResponse dr = 
txnResponse.getDeleteResponses().get(deletedIdx++);
+                            if (dr.getDeleted() == 0) {
+                                del.getFuture().completeExceptionally(new 
MetadataStoreException.NotFoundException());
+                            } else {
+                                del.getFuture().complete(null);
+                            }
+                            break;
+                        }
+                        case GET_CHILDREN: {
+                            OpGetChildren getChildren = op.asGetChildren();
+                            GetResponse gr = 
txnResponse.getGetResponses().get(getIdx++);
+                            String basePath = getChildren.getPath() + "/";
+
+                            Set<String> children = gr.getKvs().stream()
+                                    .map(kv -> 
kv.getKey().toString(StandardCharsets.UTF_8))
+                                    .map(p -> p.replace(basePath, ""))
+                                    // Only return first-level children
+                                    .map(k -> k.split("/", 2)[0])
+                                    
.collect(Collectors.toCollection(TreeSet::new));
+
+                            getChildren.getFuture().complete(new 
ArrayList<>(children));
+                        }
+                    }
+                }
+            }).exceptionally(ex -> {
+                Throwable cause = ex.getCause();
+                if (cause instanceof ExecutionException || cause instanceof 
CompletionException) {
+                    cause = cause.getCause();
+                }
+                if (ops.size() > 1 && cause instanceof StatusRuntimeException) 
{
+                    Status.Code code = ((StatusRuntimeException) 
cause).getStatus().getCode();
+                    if (
+                        // This could be caused by having repeated keys in the 
batch, retry individually
+                            code == Status.Code.INVALID_ARGUMENT ||
+
+                                    // We might have exceeded the max-frame 
size for the respons
+                                    code == Status.Code.RESOURCE_EXHAUSTED
+                    ) {
+                        ops.forEach(o -> 
batchOperation(Collections.singletonList(o)));
+                    }
+                } else {
+                    log.warn("Failed to commit: {}", cause.getMessage());
+                    ops.forEach(o -> o.getFuture().completeExceptionally(ex));
+                }
+                return null;
+            });
+        } catch (Throwable t) {
+            log.warn("Error in committing batch: {}", t.getMessage());
+        }
+    }
+
+    private synchronized CompletableFuture<Void> createLease(boolean 
retryOnFailure) {
+        CompletableFuture<Void> future = 
client.getLeaseClient().grant(leaseTTLSeconds)
+                .thenAccept(lease -> {
+                    synchronized (this) {
+                        this.leaseId = lease.getID();
+
+                        if (leaseClient != null) {
+                            leaseClient.close();
+                        }
+                        this.leaseClient =
+                                this.client.getLeaseClient()
+                                        .keepAlive(leaseId, new 
StreamObserver<LeaseKeepAliveResponse>() {
+                                            @Override
+                                            public void 
onNext(LeaseKeepAliveResponse leaseKeepAliveResponse) {
+                                                if (log.isDebugEnabled()) {
+                                                    log.debug("On next: {}", 
leaseKeepAliveResponse);
+                                                }
+                                            }
+
+                                            @Override
+                                            public void onError(Throwable 
throwable) {
+                                                log.warn("Lease client error 
:", throwable);
+                                                
receivedSessionEvent(SessionEvent.SessionLost);
+                                            }
+
+                                            @Override
+                                            public void onCompleted() {
+                                                log.info("Etcd lease has 
expired");
+                                                
receivedSessionEvent(SessionEvent.SessionLost);
+                                            }
+                                        });
+                    }
+                });
+
+        if (retryOnFailure) {
+            future.exceptionally(ex -> {
+                log.warn("Failed to create Etcd lease. Retrying later", ex);
+                executor.schedule(() -> {
+                    createLease(true);
+                }, 1, TimeUnit.SECONDS);
+                return null;
+            });
+        }
+
+        return future;
+    }
+
+    private void handleWatchResponse(WatchResponse watchResponse) {
+        watchResponse.getEvents().forEach(we -> {
+            String path = 
we.getKeyValue().getKey().toString(StandardCharsets.UTF_8);
+            String parent = parent(path);
+            if (we.getEventType() == WatchEvent.EventType.PUT) {
+                if (we.getKeyValue().getVersion() == 1) {
+                    receivedNotification(new 
Notification(NotificationType.Created, path));
+
+                    if (parent != null) {
+                        receivedNotification(new 
Notification(NotificationType.ChildrenChanged, parent));
+                    }
+                } else {
+                    receivedNotification(new 
Notification(NotificationType.Modified, path));
+                }
+            } else if (we.getEventType() == WatchEvent.EventType.DELETE) {
+                receivedNotification(new 
Notification(NotificationType.Deleted, path));
+                if (parent != null) {
+                    receivedNotification(new 
Notification(NotificationType.ChildrenChanged, parent));
+                }
+            }
+        });
+    }
+
+    @Override
+    protected void receivedSessionEvent(SessionEvent event) {
+        if (event == SessionEvent.SessionReestablished) {
+            // Re-create the lease before notifying that we are reconnected
+            createLease(true)
+                    .thenRun(() -> {
+                        super.receivedSessionEvent(event);

Review comment:
       > * When we don't kill the broker, we re-establish a new lease and we 
keep trying doing that indefinitely
   
   My point is, in this case,  if we finally re-establish the new lease (after 
some retries), is it ok that we don't call `super.receivedSessionEvent(event)`? 
 Current CompletableFuture returned by `createLease ` only tracks the first try.

##########
File path: 
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdSessionWatcher.java
##########
@@ -0,0 +1,161 @@
+/**
+ * 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;
+
+import static 
org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables;
+import io.etcd.jetcd.ByteSequence;
+import io.etcd.jetcd.Client;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Consumer;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.metadata.api.extended.SessionEvent;
+
+/**
+ * Monitor the ETCd session state every few seconds and send notifications
+ */
+@Slf4j
+public class EtcdSessionWatcher implements AutoCloseable {
+    private final Client client;
+
+    private SessionEvent currentStatus;
+    private final Consumer<SessionEvent> sessionListener;
+
+    // Maximum time to wait for Etcd lease to be re-connected to quorum (set 
to 5/6 of SessionTimeout)
+    private final long monitorTimeoutMillis;
+
+    // Interval at which we check the state of the zk session (set to 1/15 of 
SessionTimeout)

Review comment:
       typo: "zk session"

##########
File path: 
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdSessionWatcher.java
##########
@@ -0,0 +1,161 @@
+/**
+ * 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;
+
+import static 
org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables;
+import io.etcd.jetcd.ByteSequence;
+import io.etcd.jetcd.Client;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Consumer;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.metadata.api.extended.SessionEvent;
+
+/**
+ * Monitor the ETCd session state every few seconds and send notifications
+ */
+@Slf4j
+public class EtcdSessionWatcher implements AutoCloseable {
+    private final Client client;
+
+    private SessionEvent currentStatus;
+    private final Consumer<SessionEvent> sessionListener;
+
+    // Maximum time to wait for Etcd lease to be re-connected to quorum (set 
to 5/6 of SessionTimeout)
+    private final long monitorTimeoutMillis;
+
+    // Interval at which we check the state of the zk session (set to 1/15 of 
SessionTimeout)
+    private final long tickTimeMillis;
+
+    private final ScheduledExecutorService scheduler;
+    private final ScheduledFuture<?> task;
+
+    private long disconnectedAt = 0;
+
+    public EtcdSessionWatcher(Client client, long sessionTimeoutMillis,
+                              Consumer<SessionEvent> sessionListener) {
+        this.client = client;
+        this.monitorTimeoutMillis = sessionTimeoutMillis * 5 / 6;
+        this.tickTimeMillis = sessionTimeoutMillis / 15;
+        this.sessionListener = sessionListener;
+
+        this.scheduler = Executors
+                .newSingleThreadScheduledExecutor(new 
DefaultThreadFactory("metadata-store-etcd-session-watcher"));
+        this.task =
+                
scheduler.scheduleAtFixedRate(catchingAndLoggingThrowables(this::checkConnectionStatus),
 tickTimeMillis,
+                        tickTimeMillis,
+                        TimeUnit.MILLISECONDS);
+        this.currentStatus = SessionEvent.SessionReestablished;
+    }
+
+    @Override
+    public void close() throws Exception {
+        task.cancel(true);
+        scheduler.shutdownNow();
+        scheduler.awaitTermination(10, TimeUnit.SECONDS);
+    }
+
+    // task that runs every TICK_TIME to check zk connection
+    private synchronized void checkConnectionStatus() {
+        try {
+            CompletableFuture<SessionEvent> future = new CompletableFuture<>();
+            
client.getKVClient().get(ByteSequence.from("/".getBytes(StandardCharsets.UTF_8)))
+                    .thenRun(() -> {
+                        future.complete(SessionEvent.Reconnected);
+                    }).exceptionally(ex -> {
+                        future.complete(SessionEvent.ConnectionLost);
+                        return null;
+                    });
+
+            SessionEvent ectdClientState;
+            try {
+                ectdClientState = future.get(tickTimeMillis, 
TimeUnit.MILLISECONDS);
+            } catch (TimeoutException e) {
+                // Consider etcd disconnection if etcd operation takes more 
than TICK_TIME
+                ectdClientState = SessionEvent.ConnectionLost;
+            }
+
+            checkState(ectdClientState);
+        } catch (RejectedExecutionException | InterruptedException e) {
+            task.cancel(true);
+        } catch (Throwable t) {
+            log.warn("Error while checking ZK connection status", t);
+        }
+    }
+
+    synchronized void setSessionInvalid() {
+        currentStatus = SessionEvent.SessionLost;
+    }
+
+    private void checkState(SessionEvent etcdlientState) {
+        switch (etcdlientState) {
+            case SessionLost:
+                if (currentStatus != SessionEvent.SessionLost) {
+                    log.error("Etcd lease has expired expired");

Review comment:
       "expired expired"

##########
File path: 
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdSessionWatcher.java
##########
@@ -0,0 +1,161 @@
+/**
+ * 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;
+
+import static 
org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables;
+import io.etcd.jetcd.ByteSequence;
+import io.etcd.jetcd.Client;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Consumer;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.metadata.api.extended.SessionEvent;
+
+/**
+ * Monitor the ETCd session state every few seconds and send notifications
+ */
+@Slf4j
+public class EtcdSessionWatcher implements AutoCloseable {
+    private final Client client;
+
+    private SessionEvent currentStatus;
+    private final Consumer<SessionEvent> sessionListener;
+
+    // Maximum time to wait for Etcd lease to be re-connected to quorum (set 
to 5/6 of SessionTimeout)
+    private final long monitorTimeoutMillis;
+
+    // Interval at which we check the state of the zk session (set to 1/15 of 
SessionTimeout)
+    private final long tickTimeMillis;
+
+    private final ScheduledExecutorService scheduler;
+    private final ScheduledFuture<?> task;
+
+    private long disconnectedAt = 0;
+
+    public EtcdSessionWatcher(Client client, long sessionTimeoutMillis,
+                              Consumer<SessionEvent> sessionListener) {
+        this.client = client;
+        this.monitorTimeoutMillis = sessionTimeoutMillis * 5 / 6;
+        this.tickTimeMillis = sessionTimeoutMillis / 15;
+        this.sessionListener = sessionListener;
+
+        this.scheduler = Executors
+                .newSingleThreadScheduledExecutor(new 
DefaultThreadFactory("metadata-store-etcd-session-watcher"));
+        this.task =
+                
scheduler.scheduleAtFixedRate(catchingAndLoggingThrowables(this::checkConnectionStatus),
 tickTimeMillis,
+                        tickTimeMillis,
+                        TimeUnit.MILLISECONDS);
+        this.currentStatus = SessionEvent.SessionReestablished;
+    }
+
+    @Override
+    public void close() throws Exception {
+        task.cancel(true);
+        scheduler.shutdownNow();
+        scheduler.awaitTermination(10, TimeUnit.SECONDS);
+    }
+
+    // task that runs every TICK_TIME to check zk connection

Review comment:
       " zk connection"

##########
File path: 
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdSessionWatcher.java
##########
@@ -0,0 +1,161 @@
+/**
+ * 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;
+
+import static 
org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables;
+import io.etcd.jetcd.ByteSequence;
+import io.etcd.jetcd.Client;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Consumer;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.metadata.api.extended.SessionEvent;
+
+/**
+ * Monitor the ETCd session state every few seconds and send notifications
+ */
+@Slf4j
+public class EtcdSessionWatcher implements AutoCloseable {
+    private final Client client;
+
+    private SessionEvent currentStatus;
+    private final Consumer<SessionEvent> sessionListener;
+
+    // Maximum time to wait for Etcd lease to be re-connected to quorum (set 
to 5/6 of SessionTimeout)
+    private final long monitorTimeoutMillis;
+
+    // Interval at which we check the state of the zk session (set to 1/15 of 
SessionTimeout)
+    private final long tickTimeMillis;
+
+    private final ScheduledExecutorService scheduler;
+    private final ScheduledFuture<?> task;
+
+    private long disconnectedAt = 0;
+
+    public EtcdSessionWatcher(Client client, long sessionTimeoutMillis,
+                              Consumer<SessionEvent> sessionListener) {
+        this.client = client;
+        this.monitorTimeoutMillis = sessionTimeoutMillis * 5 / 6;
+        this.tickTimeMillis = sessionTimeoutMillis / 15;
+        this.sessionListener = sessionListener;
+
+        this.scheduler = Executors
+                .newSingleThreadScheduledExecutor(new 
DefaultThreadFactory("metadata-store-etcd-session-watcher"));
+        this.task =
+                
scheduler.scheduleAtFixedRate(catchingAndLoggingThrowables(this::checkConnectionStatus),
 tickTimeMillis,
+                        tickTimeMillis,
+                        TimeUnit.MILLISECONDS);
+        this.currentStatus = SessionEvent.SessionReestablished;
+    }
+
+    @Override
+    public void close() throws Exception {
+        task.cancel(true);
+        scheduler.shutdownNow();
+        scheduler.awaitTermination(10, TimeUnit.SECONDS);
+    }
+
+    // task that runs every TICK_TIME to check zk connection
+    private synchronized void checkConnectionStatus() {
+        try {
+            CompletableFuture<SessionEvent> future = new CompletableFuture<>();
+            
client.getKVClient().get(ByteSequence.from("/".getBytes(StandardCharsets.UTF_8)))
+                    .thenRun(() -> {
+                        future.complete(SessionEvent.Reconnected);
+                    }).exceptionally(ex -> {
+                        future.complete(SessionEvent.ConnectionLost);
+                        return null;
+                    });
+
+            SessionEvent ectdClientState;
+            try {
+                ectdClientState = future.get(tickTimeMillis, 
TimeUnit.MILLISECONDS);
+            } catch (TimeoutException e) {
+                // Consider etcd disconnection if etcd operation takes more 
than TICK_TIME
+                ectdClientState = SessionEvent.ConnectionLost;
+            }
+
+            checkState(ectdClientState);
+        } catch (RejectedExecutionException | InterruptedException e) {
+            task.cancel(true);
+        } catch (Throwable t) {
+            log.warn("Error while checking ZK connection status", t);

Review comment:
       " ZK connection"

##########
File path: 
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java
##########
@@ -0,0 +1,423 @@
+/**
+ * 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;
+
+import io.etcd.jetcd.ByteSequence;
+import io.etcd.jetcd.Client;
+import io.etcd.jetcd.KV;
+import io.etcd.jetcd.KeyValue;
+import io.etcd.jetcd.Txn;
+import io.etcd.jetcd.kv.DeleteResponse;
+import io.etcd.jetcd.kv.GetResponse;
+import io.etcd.jetcd.kv.PutResponse;
+import io.etcd.jetcd.lease.LeaseKeepAliveResponse;
+import io.etcd.jetcd.op.Cmp;
+import io.etcd.jetcd.op.CmpTarget;
+import io.etcd.jetcd.op.Op;
+import io.etcd.jetcd.options.DeleteOption;
+import io.etcd.jetcd.options.GetOption;
+import io.etcd.jetcd.options.PutOption;
+import io.etcd.jetcd.options.WatchOption;
+import io.etcd.jetcd.support.CloseableClient;
+import io.etcd.jetcd.watch.WatchEvent;
+import io.etcd.jetcd.watch.WatchResponse;
+import io.grpc.Status;
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+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.MetadataStoreException;
+import org.apache.pulsar.metadata.api.Notification;
+import org.apache.pulsar.metadata.api.NotificationType;
+import org.apache.pulsar.metadata.api.Stat;
+import org.apache.pulsar.metadata.api.extended.CreateOption;
+import org.apache.pulsar.metadata.api.extended.SessionEvent;
+import org.apache.pulsar.metadata.impl.batching.AbstractBatchedMetadataStore;
+import org.apache.pulsar.metadata.impl.batching.MetadataOp;
+import org.apache.pulsar.metadata.impl.batching.OpDelete;
+import org.apache.pulsar.metadata.impl.batching.OpGet;
+import org.apache.pulsar.metadata.impl.batching.OpGetChildren;
+import org.apache.pulsar.metadata.impl.batching.OpPut;
+import org.apache.zookeeper.AddWatchMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+
+@Slf4j
+public class EtcdMetadataStore extends AbstractBatchedMetadataStore {
+
+    static final String ETCD_SCHEME_IDENTIFIER = "etcd:";
+
+    private final int leaseTTLSeconds;
+    private final Client client;
+    private final KV kv;
+    private volatile long leaseId;
+    private volatile CloseableClient leaseClient;
+    private final EtcdSessionWatcher sessionWatcher;
+
+    public EtcdMetadataStore(String metadataURL, MetadataStoreConfig conf, 
boolean enableSessionWatcher)
+            throws MetadataStoreException {
+        super(conf);
+
+        this.leaseTTLSeconds = conf.getSessionTimeoutMillis() / 1000;
+        String etcdUrl = metadataURL.replaceFirst(ETCD_SCHEME_IDENTIFIER, "");
+
+        try {
+            this.client = Client.builder().endpoints(etcdUrl).build();
+            this.kv = client.getKVClient();
+            this.client.getWatchClient().watch(ByteSequence.from("\0", 
StandardCharsets.UTF_8),
+                    WatchOption.newBuilder()
+                            .withPrefix(ByteSequence.from("/", 
StandardCharsets.UTF_8))
+                            .build(), this::handleWatchResponse);
+            this.sessionWatcher =
+                    new EtcdSessionWatcher(client, 
conf.getSessionTimeoutMillis(), this::receivedSessionEvent);
+
+            // Ensure the lease is created when we start
+            this.createLease(false).join();
+        } catch (Exception e) {
+            throw new MetadataStoreException(e);
+        }
+    }
+
+    @Override
+    public void close() throws Exception {
+        super.close();
+
+        if (sessionWatcher != null) {
+            sessionWatcher.close();
+        }
+
+        leaseClient.close();
+        client.getLeaseClient().revoke(leaseId);
+
+        kv.close();
+        client.close();
+    }
+
+    private static final GetOption EXISTS_GET_OPTION = 
GetOption.newBuilder().withCountOnly(true).build();
+    private static final GetOption SINGLE_GET_OPTION = 
GetOption.newBuilder().withLimit(1).build();
+
+    @Override
+    protected CompletableFuture<Boolean> existsFromStore(String path) {
+        return kv.get(ByteSequence.from(path, StandardCharsets.UTF_8), 
EXISTS_GET_OPTION)
+                .thenApply(gr -> gr.getCount() == 1);
+    }
+
+    @Override
+    protected CompletableFuture<Stat> storePut(String path, byte[] data, 
Optional<Long> optExpectedVersion,
+                                               EnumSet<CreateOption> options) {
+        if (!options.contains(CreateOption.Sequential)) {
+            return super.storePut(path, data, optExpectedVersion, options);
+        } else {
+            // First get the version from parent
+            String parent = parent(path);
+            return super.storePut(parent, new byte[0], Optional.empty(), 
EnumSet.noneOf(CreateOption.class))
+                    // Then create the unique key with the version added in 
the path
+                    .thenCompose(stat -> super.storePut(path + 
stat.getVersion(), data, optExpectedVersion, options));
+        }
+    }
+
+    @Override
+    protected void batchOperation(List<MetadataOp> ops) {
+        try {
+            Txn txn = kv.txn();
+
+            // First, set all the conditions
+            ops.forEach(op -> {
+                switch (op.getType()) {
+                    case PUT: {
+                        OpPut put = op.asPut();
+                        ByteSequence key = ByteSequence.from(put.getPath(), 
StandardCharsets.UTF_8);
+                        if (put.getOptExpectedVersion().isPresent()) {
+                            long expectedVersion = 
put.getOptExpectedVersion().get();
+                            if (expectedVersion == -1L) {
+                                // Check that key does not exist
+                                txn.If(new Cmp(key, Cmp.Op.EQUAL, 
CmpTarget.createRevision(0)));
+                            } else {
+                                txn.If(new Cmp(key, Cmp.Op.EQUAL, 
CmpTarget.version(expectedVersion + 1)));
+                            }
+                        }
+                        break;
+                    }
+                    case DELETE: {
+                        OpDelete del = op.asDelete();
+                        ByteSequence key = ByteSequence.from(del.getPath(), 
StandardCharsets.UTF_8);
+                        if (del.getOptExpectedVersion().isPresent()) {
+                            txn.If(new Cmp(key, Cmp.Op.EQUAL,
+                                    
CmpTarget.version(del.getOptExpectedVersion().get() + 1)));
+                        }
+                        break;
+                    }
+                }
+            });
+
+            // Then the requests
+            ops.forEach(op -> {
+                switch (op.getType()) {
+                    case GET: {
+                        txn.Then(
+                                Op.get(ByteSequence.from(op.asGet().getPath(), 
StandardCharsets.UTF_8),
+                                        SINGLE_GET_OPTION));
+                        break;
+                    }
+                    case PUT: {
+                        OpPut put = op.asPut();
+                        ByteSequence key = ByteSequence.from(put.getPath(), 
StandardCharsets.UTF_8);
+                        if (!put.getFuture().isDone()) {
+                            PutOption.Builder b = PutOption.newBuilder()
+                                    .withPrevKV();
+
+                            if (put.isEphemeral()) {
+                                b.withLeaseId(leaseId);
+                            }
+
+                            txn.Then(Op.put(key, 
ByteSequence.from(put.getData()), b.build()));
+                        }
+                        break;
+                    }
+                    case DELETE: {
+                        OpDelete del = op.asDelete();
+                        ByteSequence key = ByteSequence.from(del.getPath(), 
StandardCharsets.UTF_8);
+                        txn.Then(Op.delete(key, DeleteOption.DEFAULT));
+                        break;
+                    }
+                    case GET_CHILDREN: {
+                        OpGetChildren opGetChildren = op.asGetChildren();
+                        String path = opGetChildren.getPath();
+
+                        ByteSequence firstKey =
+                                ByteSequence.from(path.equals("/") ? path : 
path + "/",
+                                        StandardCharsets.UTF_8);
+                        ByteSequence lastKey = 
ByteSequence.from(path.equals("/") ? "0" : path + "0",
+                                StandardCharsets.UTF_8); // '0' is 
lexicographically just after '/'
+
+                        txn.Then(Op.get(firstKey, GetOption.newBuilder()
+                                .withKeysOnly(true)
+                                .withSortField(GetOption.SortTarget.KEY)
+                                .withSortOrder(GetOption.SortOrder.ASCEND)
+                                .withRange(lastKey)
+                                .build()));
+                        break;
+                    }
+                }
+            });
+
+            txn.commit().thenAccept(txnResponse -> {
+                if (!txnResponse.isSucceeded()) {
+                    if (ops.size() > 1) {
+                        // Retry individually
+                        ops.forEach(o -> 
batchOperation(Collections.singletonList(o)));
+                    } else {
+                        ops.get(0).getFuture()
+                                .completeExceptionally(new 
MetadataStoreException.BadVersionException("Bad version"));
+                    }
+                    return;
+                }
+
+                int getIdx = 0;
+                int deletedIdx = 0;
+                int putIdx = 0;
+                for (MetadataOp op : ops) {
+                    switch (op.getType()) {
+                        case GET: {
+                            OpGet get = op.asGet();
+                            GetResponse gr = 
txnResponse.getGetResponses().get(getIdx++);
+                            if (gr.getCount() == 0) {
+                                get.getFuture().complete(Optional.empty());
+                            } else {
+                                KeyValue kv = gr.getKvs().get(0);
+                                boolean isEphemeral = kv.getLease() != 0;
+                                boolean createdBySelf = kv.getLease() == 
leaseId;
+                                get.getFuture().complete(Optional.of(
+                                                new GetResult(
+                                                        
kv.getValue().getBytes(),
+                                                        new 
Stat(get.getPath(), kv.getVersion() - 1, 0, 0, isEphemeral,
+                                                                createdBySelf)
+                                                )
+                                        )
+                                );
+                            }
+                            break;
+                        }
+                        case PUT: {
+                            OpPut put = op.asPut();
+                            PutResponse pr = 
txnResponse.getPutResponses().get(putIdx++);
+                            KeyValue prevKv = pr.getPrevKv();
+                            if (prevKv == null) {
+                                put.getFuture().complete(new 
Stat(put.getPath(),
+                                        0, 0, 0, put.isEphemeral(), true));
+                            } else {
+                                put.getFuture().complete(new 
Stat(put.getPath(),
+                                        prevKv.getVersion(), 0, 0, 
put.isEphemeral(), true));
+                            }
+
+                            break;
+                        }
+                        case DELETE: {
+                            OpDelete del = op.asDelete();
+                            DeleteResponse dr = 
txnResponse.getDeleteResponses().get(deletedIdx++);
+                            if (dr.getDeleted() == 0) {
+                                del.getFuture().completeExceptionally(new 
MetadataStoreException.NotFoundException());
+                            } else {
+                                del.getFuture().complete(null);
+                            }
+                            break;
+                        }
+                        case GET_CHILDREN: {
+                            OpGetChildren getChildren = op.asGetChildren();
+                            GetResponse gr = 
txnResponse.getGetResponses().get(getIdx++);
+                            String basePath = getChildren.getPath() + "/";
+
+                            Set<String> children = gr.getKvs().stream()
+                                    .map(kv -> 
kv.getKey().toString(StandardCharsets.UTF_8))
+                                    .map(p -> p.replace(basePath, ""))
+                                    // Only return first-level children
+                                    .map(k -> k.split("/", 2)[0])
+                                    
.collect(Collectors.toCollection(TreeSet::new));
+
+                            getChildren.getFuture().complete(new 
ArrayList<>(children));
+                        }
+                    }
+                }
+            }).exceptionally(ex -> {
+                Throwable cause = ex.getCause();
+                if (cause instanceof ExecutionException || cause instanceof 
CompletionException) {
+                    cause = cause.getCause();
+                }
+                if (ops.size() > 1 && cause instanceof StatusRuntimeException) 
{
+                    Status.Code code = ((StatusRuntimeException) 
cause).getStatus().getCode();
+                    if (
+                        // This could be caused by having repeated keys in the 
batch, retry individually
+                            code == Status.Code.INVALID_ARGUMENT ||
+
+                                    // We might have exceeded the max-frame 
size for the respons
+                                    code == Status.Code.RESOURCE_EXHAUSTED
+                    ) {
+                        ops.forEach(o -> 
batchOperation(Collections.singletonList(o)));
+                    }
+                } else {
+                    log.warn("Failed to commit: {}", cause.getMessage());
+                    ops.forEach(o -> o.getFuture().completeExceptionally(ex));
+                }
+                return null;
+            });
+        } catch (Throwable t) {
+            log.warn("Error in committing batch: {}", t.getMessage());
+        }
+    }
+
+    private synchronized CompletableFuture<Void> createLease(boolean 
retryOnFailure) {
+        CompletableFuture<Void> future = 
client.getLeaseClient().grant(leaseTTLSeconds)
+                .thenAccept(lease -> {
+                    synchronized (this) {
+                        this.leaseId = lease.getID();
+
+                        if (leaseClient != null) {
+                            leaseClient.close();
+                        }
+                        this.leaseClient =
+                                this.client.getLeaseClient()
+                                        .keepAlive(leaseId, new 
StreamObserver<LeaseKeepAliveResponse>() {
+                                            @Override
+                                            public void 
onNext(LeaseKeepAliveResponse leaseKeepAliveResponse) {
+                                                if (log.isDebugEnabled()) {
+                                                    log.debug("On next: {}", 
leaseKeepAliveResponse);
+                                                }
+                                            }
+
+                                            @Override
+                                            public void onError(Throwable 
throwable) {
+                                                log.warn("Lease client error 
:", throwable);
+                                                
receivedSessionEvent(SessionEvent.SessionLost);
+                                            }
+
+                                            @Override
+                                            public void onCompleted() {
+                                                log.info("Etcd lease has 
expired");
+                                                
receivedSessionEvent(SessionEvent.SessionLost);
+                                            }
+                                        });
+                    }
+                });
+
+        if (retryOnFailure) {
+            future.exceptionally(ex -> {
+                log.warn("Failed to create Etcd lease. Retrying later", ex);
+                executor.schedule(() -> {
+                    createLease(true);
+                }, 1, TimeUnit.SECONDS);
+                return null;
+            });
+        }
+
+        return future;
+    }
+
+    private void handleWatchResponse(WatchResponse watchResponse) {
+        watchResponse.getEvents().forEach(we -> {
+            String path = 
we.getKeyValue().getKey().toString(StandardCharsets.UTF_8);
+            String parent = parent(path);
+            if (we.getEventType() == WatchEvent.EventType.PUT) {
+                if (we.getKeyValue().getVersion() == 1) {
+                    receivedNotification(new 
Notification(NotificationType.Created, path));
+
+                    if (parent != null) {
+                        receivedNotification(new 
Notification(NotificationType.ChildrenChanged, parent));
+                    }
+                } else {
+                    receivedNotification(new 
Notification(NotificationType.Modified, path));
+                }
+            } else if (we.getEventType() == WatchEvent.EventType.DELETE) {
+                receivedNotification(new 
Notification(NotificationType.Deleted, path));
+                if (parent != null) {
+                    receivedNotification(new 
Notification(NotificationType.ChildrenChanged, parent));
+                }
+            }
+        });
+    }
+
+    @Override
+    protected void receivedSessionEvent(SessionEvent event) {
+        if (event == SessionEvent.SessionReestablished) {
+            // Re-create the lease before notifying that we are reconnected
+            createLease(true)
+                    .thenRun(() -> {
+                        super.receivedSessionEvent(event);

Review comment:
       I see, thx~




-- 
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-unsubscr...@pulsar.apache.org

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


Reply via email to