@Inject
  public SingleServerInventoryView(
      final ZkPathsConfig zkPaths,
      final CuratorFramework curator,
      final ObjectMapper jsonMapper,
      final Predicate<Pair<DruidServerMetadata, DataSegment>> defaultFilter) {
    super(
        log,
        zkPaths.getAnnouncementsPath(),
        zkPaths.getServedSegmentsPath(),
        curator,
        jsonMapper,
        new TypeReference<DataSegment>() {});

    Preconditions.checkNotNull(defaultFilter);
    this.defaultFilter = defaultFilter;
  }
  @Test
  public void testRun() throws Exception {
    TestingCluster localCluster = new TestingCluster(1);
    localCluster.start();

    CuratorFramework localCf =
        CuratorFrameworkFactory.builder()
            .connectString(localCluster.getConnectString())
            .retryPolicy(new ExponentialBackoffRetry(1, 10))
            .compressionProvider(new PotentiallyGzippedCompressionProvider(false))
            .build();
    localCf.start();

    TestingCluster remoteCluster = new TestingCluster(1);
    remoteCluster.start();

    CuratorFramework remoteCf =
        CuratorFrameworkFactory.builder()
            .connectString(remoteCluster.getConnectString())
            .retryPolicy(new ExponentialBackoffRetry(1, 10))
            .compressionProvider(new PotentiallyGzippedCompressionProvider(false))
            .build();
    remoteCf.start();

    ObjectMapper jsonMapper = new DefaultObjectMapper();
    DruidClusterBridgeConfig config =
        new DruidClusterBridgeConfig() {
          @Override
          public String getTier() {
            return DruidServer.DEFAULT_TIER;
          }

          @Override
          public Duration getStartDelay() {
            return new Duration(0);
          }

          @Override
          public Duration getPeriod() {
            return new Duration(Long.MAX_VALUE);
          }

          @Override
          public String getBrokerServiceName() {
            return "testz0rz";
          }

          @Override
          public int getPriority() {
            return 0;
          }
        };

    ScheduledExecutorFactory factory = ScheduledExecutors.createFactory(new Lifecycle());

    DruidNode me = new DruidNode("me", "localhost", 8080);

    AtomicReference<LeaderLatch> leaderLatch =
        new AtomicReference<>(new LeaderLatch(localCf, "test"));

    ZkPathsConfig zkPathsConfig =
        new ZkPathsConfig() {
          @Override
          public String getZkBasePath() {
            return "/druid";
          }
        };
    DruidServerMetadata metadata =
        new DruidServerMetadata("test", "localhost", 1000, "bridge", DruidServer.DEFAULT_TIER, 0);
    DbSegmentPublisher dbSegmentPublisher = EasyMock.createMock(DbSegmentPublisher.class);
    EasyMock.replay(dbSegmentPublisher);
    DatabaseSegmentManager databaseSegmentManager =
        EasyMock.createMock(DatabaseSegmentManager.class);
    EasyMock.replay(databaseSegmentManager);
    ServerView serverView = EasyMock.createMock(ServerView.class);
    EasyMock.replay(serverView);

    BridgeZkCoordinator bridgeZkCoordinator =
        new BridgeZkCoordinator(
            jsonMapper,
            zkPathsConfig,
            new SegmentLoaderConfig(),
            metadata,
            remoteCf,
            dbSegmentPublisher,
            databaseSegmentManager,
            serverView);

    Announcer announcer = new Announcer(remoteCf, Executors.newSingleThreadExecutor());
    announcer.start();
    announcer.announce(
        zkPathsConfig.getAnnouncementsPath() + "/" + me.getHost(),
        jsonMapper.writeValueAsBytes(me));

    BatchDataSegmentAnnouncer batchDataSegmentAnnouncer =
        EasyMock.createMock(BatchDataSegmentAnnouncer.class);
    BatchServerInventoryView batchServerInventoryView =
        EasyMock.createMock(BatchServerInventoryView.class);
    EasyMock.expect(batchServerInventoryView.getInventory())
        .andReturn(
            Arrays.asList(
                new DruidServer("1", "localhost", 117, "historical", DruidServer.DEFAULT_TIER, 0),
                new DruidServer("2", "localhost", 1, "historical", DruidServer.DEFAULT_TIER, 0)));
    batchServerInventoryView.registerSegmentCallback(
        EasyMock.<Executor>anyObject(), EasyMock.<ServerView.SegmentCallback>anyObject());
    batchServerInventoryView.registerServerCallback(
        EasyMock.<Executor>anyObject(), EasyMock.<ServerView.ServerCallback>anyObject());
    EasyMock.expectLastCall();
    batchServerInventoryView.start();
    EasyMock.expectLastCall();
    batchServerInventoryView.stop();
    EasyMock.expectLastCall();
    EasyMock.replay(batchServerInventoryView);

    DruidClusterBridge bridge =
        new DruidClusterBridge(
            jsonMapper,
            config,
            factory,
            me,
            localCf,
            leaderLatch,
            bridgeZkCoordinator,
            announcer,
            batchDataSegmentAnnouncer,
            batchServerInventoryView);

    bridge.start();

    int retry = 0;
    while (!bridge.isLeader()) {
      if (retry > 5) {
        throw new ISE("Unable to become leader");
      }

      Thread.sleep(100);
      retry++;
    }

    String path = "/druid/announcements/localhost:8080";
    retry = 0;
    while (remoteCf.checkExists().forPath(path) == null) {
      if (retry > 5) {
        throw new ISE("Unable to announce");
      }

      Thread.sleep(100);
      retry++;
    }

    boolean verified = verifyUpdate(jsonMapper, path, remoteCf);
    retry = 0;
    while (!verified) {
      if (retry > 5) {
        throw new ISE("No updates to bridge node occurred");
      }

      Thread.sleep(100);
      retry++;

      verified = verifyUpdate(jsonMapper, path, remoteCf);
    }

    announcer.stop();
    bridge.stop();

    remoteCf.close();
    remoteCluster.close();
    localCf.close();
    localCluster.close();

    EasyMock.verify(batchServerInventoryView);
    EasyMock.verify(dbSegmentPublisher);
    EasyMock.verify(databaseSegmentManager);
    EasyMock.verify(serverView);
  }