001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.zookeeper;
019
020import org.apache.zookeeper.WatchedEvent;
021import org.apache.zookeeper.Watcher;
022
023/**
024 * Placeholder of a watcher which might be triggered before the instance is not yet created.
025 * <p>
026 * {@code ZooKeeper} starts its event thread within its constructor (and that is an anti-pattern),
027 * and the watcher passed to the constructor might be called back by the event thread before you get
028 * the instance of {@code ZooKeeper} from the constructor. If your watcher calls methods of
029 * {@code ZooKeeper}, pass this placeholder to the constructor of the {@code ZooKeeper}, create your
030 * watcher using the instance of {@code ZooKeeper}, and then call the method
031 * {@code PendingWatcher.prepare}.
032 */
033class PendingWatcher implements Watcher {
034  private final InstancePending<Watcher> pending = new InstancePending<>();
035
036  @Override
037  public void process(WatchedEvent event) {
038    pending.get().process(event);
039  }
040
041  /**
042   * Associates the substantial watcher of processing events. This method should be called once, and
043   * {@code watcher} should be non-null. This method is expected to call as soon as possible because
044   * the event processing, being invoked by the ZooKeeper event thread, is uninterruptibly blocked
045   * until this method is called.
046   */
047  void prepare(Watcher watcher) {
048    pending.prepare(watcher);
049  }
050}